├── .gitignore ├── README.md ├── pom.xml ├── spark-hyperx.iml └── src └── main └── scala └── org └── apache └── spark └── hyperx ├── HyperPregel.scala ├── Hyperedge.scala ├── HyperedgeDirection.scala ├── HyperedgeRDD.scala ├── HyperedgeTuple.scala ├── Hypergraph.scala ├── HypergraphKryoRegistrator.scala ├── HypergraphLoader.scala ├── HypergraphOps.scala ├── VertexRDD.scala ├── exp ├── Pregel.scala └── Profiler.scala ├── impl ├── FlatHyperedgePartition.scala ├── FlatHyperedgePartitionBuilder.scala ├── FlatHyperedgeTupleIterator.scala ├── HyperedgePartition.scala ├── HyperedgePartitionBuilder.scala ├── HyperedgeTupleIterator.scala ├── HypergraphImpl.scala ├── ReplicatedVertexView.scala ├── RoutingTablePartition.scala ├── Serializers.scala ├── ShippableVertexPartition.scala ├── VertexPartition.scala ├── VertexPartitionBase.scala ├── VertexPartitionBaseOps.scala └── package.scala ├── lib ├── Analytics.scala ├── BetweennessCentrality.scala ├── LabelPropagation.scala ├── LabelPropagationPartition.scala ├── Linalg.scala ├── MllibRDDFunctions.scala ├── RandomWalk.scala ├── SlidingRDD.scala └── SpectralLearning.scala ├── package.scala ├── partition ├── AwetoPartition.scala ├── BipartitePartition.scala ├── GreedyPartition.scala ├── PartitionStrategy.scala ├── PlainPartition.scala ├── ReplicaPartition.scala ├── VertexPartitioner.scala └── package.scala └── util ├── BytecodeUtils.scala ├── HyperUtils.scala ├── collection ├── HyperXLongBitSet.scala ├── HyperXOpenHashMap.scala ├── HyperXOpenHashSet.scala └── HyperXPrimitiveVector.scala └── package.scala /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | ./.idea/ 4 | ./.idea/* 5 | 6 | # sbt specific 7 | .cache/ 8 | .history/ 9 | .lib/ 10 | dist/* 11 | target/ 12 | lib_managed/ 13 | src_managed/ 14 | project/boot/ 15 | project/plugins/project/ 16 | 17 | # Scala-IDE specific 18 | .scala_dependencies 19 | .worksheet 20 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | HyperX 2 | ====== 3 | 4 | A scalable framework for hypergraph processing and learning algorithms. HyperX is built upon Apache Spark and inspired by its graph counterpart, GraphX. 5 | 6 | When processing a hypergraph (where an edge contains arbitrary number of vertices), instead of converting the hypergraph to a bipartite and employing GraphX to do the tricks, HyperX directly operates on a distributed hypergraph representation. By carefully optimizing the hypergraph partitioning strategies, the preliminary exprimental results show that HyperX is able to achieve a 49 speedup factor on the hypergraph random walks upon the bipartite GraphX solution. 7 | 8 | A paper describing the details is now under review for ICDM 2015. A technical report can be found at http://iojin.com/resources/hyperx_report.pdf. 9 | 10 | 11 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | org.apache.spark 8 | spark-hyperx 9 | jar 10 | Spark HyperX 11 | http://github.com/jinhuang/hyperx 12 | 0.0.1-SNAPSHOT 13 | 14 | 15 | 1.6 16 | 2.10.4 17 | 2.10 18 | 2.0.1 19 | 1.1.0 20 | 21 | 22 | 23 | 24 | org.scala-lang 25 | scala-library 26 | 2.10.4 27 | compile 28 | 29 | 30 | 31 | com.google.guava 32 | guava 33 | 14.0.1 34 | compile 35 | 36 | 37 | 38 | org.apache.spark 39 | spark-core_${scala.binary.version} 40 | ${spark.version} 41 | compile 42 | 43 | 44 | 45 | org.apache.spark 46 | spark-mllib_${scala.binary.version} 47 | ${spark.version} 48 | compile 49 | 50 | 51 | org.apache.spark 52 | spark-graphx_${scala.binary.version} 53 | ${spark.version} 54 | provided 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | org.apache.maven.plugins 63 | maven-enforcer-plugin 64 | 1.3.1 65 | 66 | 67 | enforce-versions 68 | 69 | enforce 70 | 71 | 72 | 73 | 74 | 3.0.4 75 | 76 | 77 | ${java.version} 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | org.codehaus.mojo 86 | build-helper-maven-plugin 87 | 1.8 88 | 89 | 90 | net.alchim31.maven 91 | scala-maven-plugin 92 | 3.2.0 93 | 94 | 95 | scala-compile-first 96 | process-resources 97 | 98 | compile 99 | 100 | 101 | 102 | scala-test-compile-first 103 | process-test-resources 104 | 105 | testCompile 106 | 107 | 108 | 109 | attach-scaladocs 110 | verify 111 | 112 | doc-jar 113 | 114 | 115 | 116 | 117 | ${scala.version} 118 | incremental 119 | true 120 | 121 | -unchecked 122 | -deprecation 123 | -feature 124 | -language:postfixOps 125 | 126 | 127 | -Xms1024m 128 | -Xmx1024m 129 | 130 | 131 | -source 132 | ${java.version} 133 | -target 134 | ${java.version} 135 | 136 | 138 | 139 | 140 | 141 | org.apache.maven.plugins 142 | maven-compiler-plugin 143 | 3.1 144 | 145 | ${java.version} 146 | ${java.version} 147 | UTF-8 148 | 1024m 149 | true 150 | 151 | 152 | 153 | org.apache.maven.plugins 154 | maven-surefire-plugin 155 | 2.17 156 | 157 | 158 | true 159 | 160 | 161 | 162 | org.scalatest 163 | scalatest-maven-plugin 164 | 1.0-RC2 165 | 166 | ${project.build.directory}/surefire-reports 167 | . 168 | ${project.build.directory}/SparkTestSuite.txt 169 | 170 | 171 | 172 | 173 | test 174 | 175 | test 176 | 177 | 178 | 179 | 180 | 181 | org.apache.maven.plugins 182 | maven-jar-plugin 183 | 2.4 184 | 185 | 186 | org.apache.maven.plugins 187 | maven-antrun-plugin 188 | 1.7 189 | 190 | 191 | org.apache.maven.plugins 192 | maven-shade-plugin 193 | 2.2 194 | 195 | 196 | org.apache.maven.plugins 197 | maven-source-plugin 198 | 2.2.1 199 | 200 | true 201 | 202 | 203 | 204 | create-source-jar 205 | 206 | jar-no-fork 207 | 208 | 209 | 210 | 211 | 212 | org.apache.maven.plugins 213 | maven-clean-plugin 214 | 2.5 215 | 216 | 217 | 218 | work 219 | 220 | 221 | checkpoint 222 | 223 | 224 | 225 | 226 | 227 | 228 | 229 | 230 | 231 | org.apache.maven.plugins 232 | maven-enforcer-plugin 233 | 234 | 235 | org.codehaus.mojo 236 | build-helper-maven-plugin 237 | 238 | 239 | add-scala-sources 240 | generate-sources 241 | 242 | add-source 243 | 244 | 245 | 246 | src/main/scala 247 | 248 | 249 | 250 | 251 | add-scala-test-sources 252 | generate-test-sources 253 | 254 | add-test-source 255 | 256 | 257 | 258 | src/test/scala 259 | 260 | 261 | 262 | 263 | 264 | 265 | net.alchim31.maven 266 | scala-maven-plugin 267 | 268 | 269 | org.apache.maven.plugins 270 | maven-source-plugin 271 | 272 | 273 | 274 | 275 | -------------------------------------------------------------------------------- /spark-hyperx.iml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | 41 | 42 | 43 | 44 | 45 | 46 | 47 | 48 | 49 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/Hyperedge.scala: -------------------------------------------------------------------------------- 1 | 2 | package org.apache.spark.hyperx 3 | 4 | import org.apache.spark.hyperx.util.HyperUtils 5 | 6 | /** 7 | * A single directed hyperedge consisting of a set of source ids, a set of 8 | * target ids, and the data associated with the hyperedge 9 | * 10 | * @tparam ED type of the hyperedge attribute 11 | * 12 | * Forked from GraphX, modified by Jin Huang 13 | */ 14 | case class Hyperedge[@specialized(Char, Int, Boolean, Byte, Long, Float, 15 | Double) ED](var srcIds: VertexSet,var dstIds: VertexSet, 16 | var attr: ED = null.asInstanceOf[ED]) 17 | extends Serializable { 18 | 19 | var id: HyperedgeId = null.asInstanceOf[HyperedgeId] 20 | 21 | def this() = { 22 | this(null.asInstanceOf[VertexSet], null.asInstanceOf[VertexSet]) 23 | } 24 | 25 | /** 26 | * Given one vertex set in the hyperedge return the other hyperedge 27 | * @param vids the vertex set one of the two sets on the edge 28 | * @return the set of the other vertex set on the edge 29 | */ 30 | def otherVertexIds(vids: VertexSet): VertexSet = 31 | if (HyperUtils.is(srcIds, vids)) dstIds 32 | else { 33 | assert(HyperUtils.is(dstIds, vids)) 34 | srcIds 35 | } 36 | 37 | def relativeDirection(vids: VertexSet): HyperedgeDirection = 38 | if (HyperUtils.is(srcIds, vids)) HyperedgeDirection.Out 39 | else { 40 | assert(HyperUtils.is(dstIds, vids)) 41 | HyperedgeDirection.In 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/HyperedgeDirection.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | /** 4 | * The direction of a directed hyperedge relative to a vertex set 5 | * 6 | * Forked from GraphX, modified by Jin Huang 7 | */ 8 | class HyperedgeDirection private(private val name: String) extends 9 | Serializable { 10 | 11 | /** 12 | * Reverse the direction of a hyperedge. 13 | */ 14 | def reverse: HyperedgeDirection = this match { 15 | case HyperedgeDirection.In => HyperedgeDirection.Out 16 | case HyperedgeDirection.Out => HyperedgeDirection.In 17 | case HyperedgeDirection.Either => HyperedgeDirection.Either 18 | case HyperedgeDirection.Both => HyperedgeDirection.Both 19 | } 20 | 21 | override def toString: String = "HyperedgeDirection." + name 22 | 23 | override def equals(o: Any) = o match { 24 | case other: HyperedgeDirection => other.name == name 25 | case _ => false 26 | } 27 | 28 | override def hashCode = name.hashCode 29 | } 30 | 31 | /** 32 | * A set of [[HyperedgeDirection]]s. 33 | */ 34 | object HyperedgeDirection { 35 | final val In = new HyperedgeDirection("In") 36 | 37 | final val Out = new HyperedgeDirection("Out") 38 | 39 | final val Either = new HyperedgeDirection("Either") 40 | 41 | final val Both = new HyperedgeDirection("Both") 42 | } 43 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/HyperedgeRDD.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | import org.apache.spark.hyperx.impl.{FlatHyperedgePartition, FlatHyperedgePartitionBuilder} 4 | import org.apache.spark.rdd.RDD 5 | import org.apache.spark.storage.StorageLevel 6 | import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} 7 | 8 | import scala.reflect.{ClassTag, classTag} 9 | 10 | /** Forked from GraphX, modified by Jin Huang */ 11 | class HyperedgeRDD[@specialized ED: ClassTag, VD: ClassTag]( 12 | // val partitionsRDD: RDD[(PartitionId, HyperedgePartition[ED, VD])], 13 | val partitionsRDD: RDD[(PartitionId, FlatHyperedgePartition[ED, VD])], 14 | val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) 15 | extends RDD[Hyperedge[ED]](partitionsRDD.context, 16 | List(new OneToOneDependency(partitionsRDD))) { 17 | 18 | partitionsRDD.setName("HyperedgeRDD") 19 | 20 | /** 21 | * Use the partitioner of the embedded RDD, if couldn't find one, 22 | * use the default partitioner 23 | */ 24 | override val partitioner = 25 | partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner 26 | (partitionsRDD))) 27 | 28 | override def compute(part: Partition, context: TaskContext): 29 | Iterator[Hyperedge[ED]] = { 30 | val p = firstParent[(PartitionId, FlatHyperedgePartition[ED, 31 | VD])].iterator(part, context) 32 | if (p.hasNext) { 33 | p.next()._2.iterator.map(_.copy()) 34 | } else { 35 | Iterator.empty 36 | } 37 | } 38 | 39 | override def collect(): Array[Hyperedge[ED]] = this.map[Hyperedge[ED]](h 40 | => h.copy()) collect() 41 | 42 | override def persist(newLevel: StorageLevel): this.type = { 43 | partitionsRDD.persist(newLevel) 44 | this 45 | } 46 | 47 | override def unpersist(blocking: Boolean = true): this.type = { 48 | partitionsRDD.unpersist(blocking) 49 | this 50 | } 51 | 52 | /** Persists the vertex partitions using `targetStorageLevel`, 53 | * which defaults to MEMORY_ONLY */ 54 | override def cache(): this.type = { 55 | partitionsRDD.persist(targetStorageLevel) 56 | this 57 | } 58 | 59 | /** 60 | * Map the values in a hyperedge partitioning preserving the structure 61 | * but changing the values. 62 | * @param f the function from a hyperedge to a new hyperedge attribute 63 | * @tparam ED2 the new hyperedge attribute type 64 | * @return a new HyperedgeRDD containing the new hyperedge attributes 65 | */ 66 | def mapValues[ED2: ClassTag](f: Hyperedge[ED] => ED2): HyperedgeRDD[ED2, 67 | VD] = { 68 | mapHyperedgePartitions((pid, part) => part.map(f)) 69 | } 70 | 71 | /** 72 | * Reverse all the hyperedges in this RDD 73 | * @return a new HyperedgeRDD containing all the hyperedge reversed 74 | */ 75 | def reverse: HyperedgeRDD[ED, VD] = 76 | mapHyperedgePartitions((pid,part) => part.reverse) 77 | 78 | /** Applies a function on each hyperedge partition, 79 | * and produces a new HyperedgeRDD with new partitions */ 80 | private[hyperx] def mapHyperedgePartitions[ED2: ClassTag, VD2: ClassTag]( 81 | // f: (PartitionId, HyperedgePartition[ED, VD]) => 82 | // HyperedgePartition[ED2, VD2]): HyperedgeRDD[ED2, VD2] = { 83 | f: (PartitionId, FlatHyperedgePartition[ED, VD]) => 84 | FlatHyperedgePartition[ED2, VD2]): HyperedgeRDD[ED2, VD2] = { 85 | this.withPartitionsRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => 86 | if (iter.hasNext) { 87 | val (pid, ep) = iter.next() 88 | Iterator(Tuple2(pid, f(pid, ep))) 89 | } else { 90 | Iterator.empty 91 | } 92 | }, preservesPartitioning = true)) 93 | } 94 | 95 | /** 96 | * Removen all hyperedges but those matching `hpred` and where both 97 | * vertices match `vpred` 98 | * @param hpred the predicate on hyperedges 99 | * @param vpred the predicate on vertices 100 | */ 101 | def filter(hpred: HyperedgeTuple[VD, ED] => Boolean, 102 | vpred: HyperAttr[VD] => Boolean): HyperedgeRDD[ED, VD] = { 103 | mapHyperedgePartitions((pid, part) => part.filter(hpred, vpred)) 104 | } 105 | 106 | /** 107 | * Inner joins this HyperedgeRDD with another HyperedgeRDD, 108 | * assuming both are partitioned using the same 109 | * PartitionStrategy 110 | * 111 | * This join is not efficient as the underlying joins between two 112 | * HyperedgePartitions 113 | * are carried out via a brutal cartesian product. 114 | * 115 | * @todo implement a more efficient join algorithm 116 | * 117 | * @param other the other HyperedgeRDD to join with 118 | * @param f the join function applied to corresponding values of `this` 119 | * and `other` 120 | * @tparam ED2 the attribute type of hyperedges in `other` 121 | * @tparam ED3 the attribute type of the joined hyperedges 122 | * @return a new HyperedgeRDD containing only hyperedges that appear in 123 | * both `this` 124 | * and `other` with values supplied by `f` 125 | */ 126 | def innerJoin[ED2: ClassTag, ED3: ClassTag] 127 | (other: HyperedgeRDD[ED2, _]) 128 | (f: (VertexSet, VertexSet, ED, ED2) => ED3) = { 129 | val ed2Tag = classTag[ED2] 130 | val ed3Tag = classTag[ED3] 131 | this.withPartitionsRDD[ED3, VD](partitionsRDD.zipPartitions(other 132 | .partitionsRDD, preservesPartitioning = true) { 133 | (thisIter, otherIter) => 134 | val (pid, thisEPart) = thisIter.next() 135 | val (_, otherEPart) = otherIter.next() 136 | Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f) 137 | (ed2Tag, ed3Tag))) 138 | }) 139 | } 140 | 141 | /** Replaces the vertex partitions while preserving all other properties 142 | * of the VertexRDD. */ 143 | private[hyperx] def withPartitionsRDD[ED2: ClassTag, VD2: ClassTag]( 144 | // partitionsRDD: RDD[(PartitionId, HyperedgePartition[ED2, VD2])]) 145 | partitionsRDD: RDD[(PartitionId, FlatHyperedgePartition[ED2, VD2])]) 146 | : HyperedgeRDD[ED2, VD2] = { 147 | new HyperedgeRDD(partitionsRDD, this.targetStorageLevel) 148 | } 149 | 150 | override protected def getPartitions: Array[Partition] = partitionsRDD 151 | .partitions 152 | 153 | /** Changes the target storage level while preserving all other 154 | * properties of the 155 | * HyperedgeRDD. Operations on the returned HyperedgeRDD will preserve 156 | * this storage 157 | * level. */ 158 | private[hyperx] def withTargetStorageLevel(tagrteStorageLevel:StorageLevel) 159 | :HyperedgeRDD[ED, VD] = { 160 | new HyperedgeRDD(this.partitionsRDD, targetStorageLevel) 161 | } 162 | } 163 | 164 | object HyperedgeRDD { 165 | 166 | /** 167 | * Create a HyperedgeRDD from a set (RDD) of hyperedges. 168 | * @param hyperedges the input hyperedges 169 | * @tparam ED the type of hyperedge attribute 170 | * @tparam VD the type of vertex attribute 171 | * @return a new HyperedgeRDD created for the input hyperedges 172 | */ 173 | def fromHyperedges[ED: ClassTag, VD: ClassTag]( 174 | hyperedges:RDD[Hyperedge[ED]]): HyperedgeRDD[ED, VD] = { 175 | // val hyperedgePartitions = hyperedges.mapPartitionsWithIndex{ 176 | // (pid,iter) => 177 | // val builder = new HyperedgePartitionBuilder[ED, VD] 178 | // iter.foreach { e => 179 | // builder.add(e.srcIds, e.dstIds, e.attr) 180 | // } 181 | // Iterator((pid, builder.toHyperedgePartition)) 182 | // } 183 | // HyperedgeRDD.fromHyperedgePartitions(hyperedgePartitions) 184 | val hyperedgePartitions = hyperedges.mapPartitionsWithIndex{(pid, iter) => 185 | val builder = new FlatHyperedgePartitionBuilder[ED, VD]() 186 | iter.zipWithIndex.foreach{h => 187 | builder.add(h._1.srcIds, h._1.dstIds, h._2, h._1.attr) 188 | } 189 | Iterator((pid, builder.toFlatHyperedgePartition)) 190 | } 191 | HyperedgeRDD.fromHyperedgePartitions(hyperedgePartitions) 192 | } 193 | 194 | /** 195 | * Create a HyperedgeRDD from previously constructed hyperedge partitions. 196 | * @param hyperedgePartitions the previously constructed hyperedge partitions 197 | * @tparam ED the type of hyperedge attribute 198 | * @tparam VD the type of vertex attribute 199 | * @return a new HyperedgeRDD created for the hyperedge partitions 200 | */ 201 | def fromHyperedgePartitions[ED: ClassTag, VD: ClassTag]( 202 | // hyperedgePartitions: RDD[(Int, HyperedgePartition[ED, VD])], 203 | hyperedgePartitions: RDD[(Int, FlatHyperedgePartition[ED, VD])], 204 | storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) 205 | : HyperedgeRDD[ED, VD] = { 206 | new HyperedgeRDD(hyperedgePartitions, storageLevel) 207 | } 208 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/HyperedgeTuple.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | import org.apache.spark.hyperx.util.HyperUtils 4 | 5 | import scala.reflect.ClassTag 6 | 7 | /** 8 | * A hyperedge tuple represents a hyperedge along with the vertex 9 | * attributes of all its neighboring vertexes 10 | * 11 | * @tparam VD the type of the vertex attribute 12 | * @tparam ED the type of the hyperedge attribute 13 | * 14 | * Forked from GraphX 2.10, modified by Jin Huang 15 | */ 16 | class HyperedgeTuple[VD: ClassTag, ED: ClassTag] { 17 | 18 | /** 19 | * The source vertex attributes 20 | */ 21 | var srcAttr: HyperAttr[VD] = _ 22 | 23 | /** 24 | * The destination vertex attributes 25 | */ 26 | var dstAttr: HyperAttr[VD] = _ 27 | 28 | 29 | /** 30 | * The hyperedge attribute 31 | */ 32 | var attr: ED = _ 33 | 34 | /** 35 | * Only for the internal usage within a partition 36 | */ 37 | var id: HyperedgeId = _ 38 | 39 | /** 40 | * Given one vertex set in the edge return the other vertex set attributes 41 | * @param vids the vertex set one of the two sets on the edge 42 | * @return the attributes for the other vertex set on the edge 43 | */ 44 | def otherVertexAttr(vids: VertexSet): HyperAttr[VD] = 45 | if (HyperUtils.is(srcAttr.keySet, vids)) dstAttr 46 | else { 47 | assert(HyperUtils.is(dstAttr.keySet, vids)); srcAttr 48 | } 49 | 50 | /** 51 | * Get the vertex attributes for the given vertex set in the edge 52 | * @param vids the vertex set of one of the two vertex sets on the edge 53 | * @return the attributes for the vertex set 54 | */ 55 | def vertexAttr(vids: VertexSet): HyperAttr[VD] = 56 | if (HyperUtils.is(srcAttr.keySet, vids)) srcAttr 57 | else { 58 | assert(HyperUtils.is(dstAttr.keySet, vids)); dstAttr 59 | } 60 | 61 | override def toString = 62 | (HyperUtils.mkString(srcAttr, ";"), 63 | HyperUtils.mkString(dstAttr, ";"), 64 | attr.toString).toString() 65 | 66 | def toTuple: (HyperAttr[VD], HyperAttr[VD], ED) = (srcAttr, dstAttr, attr) 67 | 68 | /** 69 | * Set the edge properties of this tuple 70 | */ 71 | protected[spark] def set(other: Hyperedge[ED]): HyperedgeTuple[VD, ED] = { 72 | srcAttr = HyperUtils.init[VD](other.srcIds) 73 | dstAttr = HyperUtils.init[VD](other.dstIds) 74 | attr = other.attr 75 | this 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/HypergraphKryoRegistrator.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | import com.esotericsoftware.kryo.Kryo 4 | import org.apache.spark.hyperx.impl._ 5 | import org.apache.spark.hyperx.partition._ 6 | import org.apache.spark.hyperx.util.collection.{HyperXOpenHashMap, HyperXOpenHashSet} 7 | import org.apache.spark.serializer.KryoRegistrator 8 | import org.apache.spark.util.BoundedPriorityQueue 9 | import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet} 10 | 11 | import scala.collection.{BitSet, immutable, mutable} 12 | 13 | /** 14 | * Register HyperX classes with Kryo 15 | */ 16 | class HypergraphKryoRegistrator extends KryoRegistrator { 17 | def registerClasses(kryo: Kryo) = { 18 | 19 | kryo.register(classOf[(VertexId, Object)]) 20 | kryo.register(classOf[BitSet]) 21 | kryo.register(classOf[VertexIdToIndexMap]) 22 | kryo.register(classOf[VertexAttributeBlock[Object]]) 23 | kryo.register(classOf[PartitionStrategy]) 24 | kryo.register(classOf[BoundedPriorityQueue[Object]]) 25 | kryo.register(classOf[HyperXOpenHashSet[Object]]) 26 | kryo.register(classOf[HyperXOpenHashMap[Object, Object]]) 27 | kryo.register(classOf[OpenHashSet[Object]]) 28 | kryo.register(classOf[OpenHashMap[Object, Object]]) 29 | kryo.register(classOf[mutable.HashMap[Object, Object]]) 30 | kryo.register(classOf[immutable.Map[Object, Object]]) 31 | kryo.register(classOf[Array[Object]]) 32 | kryo.register(classOf[Array[Int]]) 33 | kryo.register(classOf[Array[Long]]) 34 | kryo.register(classOf[Array[Double]]) 35 | kryo.register(classOf[(Any, Any, Any)]) 36 | kryo.register(classOf[HyperedgePartition[Object, Object]]) 37 | kryo.register(classOf[FlatHyperedgePartition[Object, Object]]) 38 | kryo.register(classOf[Hyperedge[Object]]) 39 | kryo.register(classOf[HyperedgeDirection]) 40 | kryo.register(classOf[ReplicatedVertexView[Object, Object]]) 41 | kryo.register(classOf[PlainPartition]) 42 | kryo.register(classOf[GreedyPartition]) 43 | kryo.register(classOf[BipartitePartition]) 44 | kryo.register(classOf[AwetoPartition]) 45 | kryo.register(classOf[ReplicaPartition]) 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/HypergraphLoader.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | import org.apache.spark.hyperx.impl.{FlatHyperedgePartition, HypergraphImpl, ShippableVertexPartition} 4 | import org.apache.spark.hyperx.partition.{PartitionStrategy, PlainPartition} 5 | import org.apache.spark.storage.StorageLevel 6 | import org.apache.spark.{Logging, SparkContext} 7 | 8 | /** 9 | * Provided utilities for loading [[Hypergraph]]s from files. 10 | * 11 | * Forked from GraphX 2.10, modified by Jin Huang 12 | */ 13 | object HypergraphLoader extends Logging { 14 | 15 | /** 16 | * Load a hypergraph from a hyperedge list formatted file where each line 17 | * contains a hyperedge as: 18 | * srcIds separator dstIds separator weight. Skip lines that begin with `#`. 19 | * 20 | * Optionally, hyperedge weight can be skipped and set to 1 by default. 21 | */ 22 | def hyperedgeListFile( 23 | sc: SparkContext, path: String, separator: String, 24 | weighted: Boolean = false, part: PartitionId, 25 | strategy: PartitionStrategy = new PlainPartition(), 26 | hyperedgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, 27 | vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) 28 | : Hypergraph[Int, Int] = { 29 | val lines = sc.textFile(path, part) 30 | val input = lines.filter{ line => 31 | val array = line.split(separator) 32 | !line.isEmpty && 33 | line(0) != '#' && 34 | line.contains(separator) && 35 | array.length >= 2 36 | } 37 | .map{line => 38 | val lineArray = line.replace(separator, ";").split(";") 39 | lineArray(0).trim + ";" + lineArray(1).trim // strip the hyperedge value for now 40 | } 41 | 42 | HypergraphImpl.fromHyperedgeList[Int, Int]( 43 | input, part, strategy, vertexStorageLevel, hyperedgeStorageLevel) 44 | 45 | } 46 | 47 | /** 48 | * Load a hypergraph from stored object files 49 | */ 50 | def hypergraphObjectFile( sc: SparkContext, vertexPath: String, 51 | hyperedgePath: String, part: PartitionId, 52 | hyperedgeLevel: StorageLevel = StorageLevel.MEMORY_ONLY, 53 | vertexLevel: StorageLevel = StorageLevel.MEMORY_ONLY) 54 | : Hypergraph[Int,Int]= { 55 | val hyperedges = 56 | sc.objectFile[(PartitionId, FlatHyperedgePartition[Int, Int])]( 57 | hyperedgePath, part) 58 | val vertices = 59 | sc.objectFile[ShippableVertexPartition[Int]](vertexPath, part) 60 | 61 | HypergraphImpl.fromPartitions[Int, Int]( 62 | hyperedges, vertices,hyperedgeLevel, vertexLevel) 63 | } 64 | 65 | /** 66 | * Load a hypergraph from a partitioned hyperedge list formatted file 67 | * where each line contains a hyperedge as: 68 | * partitionId separator srcIds separator dstIds separator weight 69 | * Skip lines that begin with `#`. 70 | * 71 | * Optionally, hyperedge weight can be skipped and set to 1 by default. 72 | */ 73 | def partitionFile(sc: SparkContext, path: String, part: PartitionId, 74 | separator: String, 75 | hyperedgeLevel: StorageLevel = StorageLevel.MEMORY_ONLY, 76 | vertexLevel: StorageLevel = StorageLevel.MEMORY_ONLY) 77 | : Hypergraph[Int, Int] = { 78 | val lines = sc.textFile(path, part) 79 | val input = lines.filter{ line => 80 | val array = line.split(separator) 81 | !line.isEmpty && line(0) != '#' && line.contains(separator) && 82 | array.length >= 3 83 | }.map(line => line.replace(separator, ";")) 84 | 85 | HypergraphImpl.fromPartitionedHyperedgeList[Int, Int]( 86 | input, part, vertexLevel, hyperedgeLevel) 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/exp/Pregel.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.exp 19 | 20 | import org.apache.spark.graphx.{EdgeTriplet, VertexId, Graph, EdgeDirection} 21 | 22 | import scala.reflect.ClassTag 23 | import org.apache.spark.Logging 24 | 25 | 26 | /** 27 | * Implements a Pregel-like bulk-synchronous message-passing API. 28 | * 29 | * Unlike the original Pregel API, the GraphX Pregel API factors the sendMessage computation over 30 | * edges, enables the message sending computation to read both vertex attributes, and constrains 31 | * messages to the graph structure. These changes allow for substantially more efficient 32 | * distributed execution while also exposing greater flexibility for graph-based computation. 33 | * 34 | * @example We can use the Pregel abstraction to implement PageRank: 35 | * {{{ 36 | * val pagerankGraph: Graph[Double, Double] = graph 37 | * // Associate the degree with each vertex 38 | * .outerJoinVertices(graph.outDegrees) { 39 | * (vid, vdata, deg) => deg.getOrElse(0) 40 | * } 41 | * // Set the weight on the edges based on the degree 42 | * .mapTriplets(e => 1.0 / e.srcAttr) 43 | * // Set the vertex attributes to the initial pagerank values 44 | * .mapVertices((id, attr) => 1.0) 45 | * 46 | * def vertexProgram(id: VertexId, attr: Double, msgSum: Double): Double = 47 | * resetProb + (1.0 - resetProb) * msgSum 48 | * def sendMessage(id: VertexId, edge: EdgeTriplet[Double, Double]): Iterator[(VertexId, Double)] = 49 | * Iterator((edge.dstId, edge.srcAttr * edge.attr)) 50 | * def messageCombiner(a: Double, b: Double): Double = a + b 51 | * val initialMessage = 0.0 52 | * // Execute Pregel for a fixed number of iterations. 53 | * Pregel(pagerankGraph, initialMessage, numIter)( 54 | * vertexProgram, sendMessage, messageCombiner) 55 | * }}} 56 | * 57 | */ 58 | object Pregel extends Logging { 59 | 60 | /** 61 | * Execute a Pregel-like iterative vertex-parallel abstraction. The 62 | * user-defined vertex-program `vprog` is executed in parallel on 63 | * each vertex receiving any inbound messages and computing a new 64 | * value for the vertex. The `sendMsg` function is then invoked on 65 | * all out-edges and is used to compute an optional message to the 66 | * destination vertex. The `mergeMsg` function is a commutative 67 | * associative function used to combine messages destined to the 68 | * same vertex. 69 | * 70 | * On the first iteration all vertices receive the `initialMsg` and 71 | * on subsequent iterations if a vertex does not receive a message 72 | * then the vertex-program is not invoked. 73 | * 74 | * This function iterates until there are no remaining messages, or 75 | * for `maxIterations` iterations. 76 | * 77 | * @tparam VD the vertex data type 78 | * @tparam ED the edge data type 79 | * @tparam A the Pregel message type 80 | * 81 | * @param graph the input graph. 82 | * 83 | * @param initialMsg the message each vertex will receive at the on 84 | * the first iteration 85 | * 86 | * @param maxIterations the maximum number of iterations to run for 87 | * 88 | * @param activeDirection the direction of edges incident to a vertex that received a message in 89 | * the previous round on which to run `sendMsg`. For example, if this is `EdgeDirection.Out`, only 90 | * out-edges of vertices that received a message in the previous round will run. The default is 91 | * `EdgeDirection.Either`, which will run `sendMsg` on edges where either side received a message 92 | * in the previous round. If this is `EdgeDirection.Both`, `sendMsg` will only run on edges where 93 | * *both* vertices received a message. 94 | * 95 | * @param vprog the user-defined vertex program which runs on each 96 | * vertex and receives the inbound message and computes a new vertex 97 | * value. On the first iteration the vertex program is invoked on 98 | * all vertices and is passed the default message. On subsequent 99 | * iterations the vertex program is only invoked on those vertices 100 | * that receive messages. 101 | * 102 | * @param sendMsg a user supplied function that is applied to out 103 | * edges of vertices that received messages in the current 104 | * iteration 105 | * 106 | * @param mergeMsg a user supplied function that takes two incoming 107 | * messages of type A and merges them into a single message of type 108 | * A. ''This function must be commutative and associative and 109 | * ideally the size of A should not increase.'' 110 | * 111 | * @return the resulting graph at the end of the computation 112 | * 113 | */ 114 | def apply[VD: ClassTag, ED: ClassTag, A: ClassTag] 115 | (graph: Graph[VD, ED], 116 | initialMsg: A, 117 | maxIterations: Int = Int.MaxValue, 118 | activeDirection: EdgeDirection = EdgeDirection.Either) 119 | (vprog: (VertexId, VD, A) => VD, 120 | sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], 121 | mergeMsg: (A, A) => A) 122 | : Graph[VD, ED] = 123 | { 124 | var g = graph.mapVertices((vid, vdata) => vprog(vid, vdata, initialMsg)).cache() 125 | // compute the messages 126 | var messages = g.mapReduceTriplets(sendMsg, mergeMsg) 127 | var activeMessages = messages.count() 128 | // Loop 129 | var prevG: Graph[VD, ED] = null 130 | var i = 0 131 | while (activeMessages > 0 && i < maxIterations) { 132 | var start = System.currentTimeMillis() 133 | // Receive the messages. Vertices that didn't get any messages do not appear in newVerts. 134 | val newVerts = g.vertices.innerJoin(messages)(vprog).cache() 135 | newVerts.count() 136 | val inner = System.currentTimeMillis() - start 137 | start = System.currentTimeMillis() 138 | // Update the graph with the new vertices. 139 | prevG = g 140 | g = g.outerJoinVertices(newVerts) { (vid, old, newOpt) => newOpt.getOrElse(old) } 141 | g.cache() 142 | val outer = System.currentTimeMillis() - start 143 | start = System.currentTimeMillis() 144 | 145 | val oldMessages = messages 146 | // Send new messages. Vertices that didn't get any messages don't appear in newVerts, so don't 147 | // get to send messages. We must cache messages so it can be materialized on the next line, 148 | // allowing us to uncache the previous iteration. 149 | messages = g.mapReduceTriplets(sendMsg, mergeMsg, Some((newVerts, activeDirection))).cache() 150 | // The call to count() materializes `messages`, `newVerts`, and the vertices of `g`. This 151 | // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the 152 | // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). 153 | activeMessages = messages.count() 154 | val mrt = System.currentTimeMillis() - start 155 | 156 | logInfo(s"Pregel finished iteration $i innner $inner outer $outer mrt $mrt with $activeMessages messages") 157 | 158 | // Unpersist the RDDs hidden by newly-materialized RDDs 159 | oldMessages.unpersist(blocking=false) 160 | newVerts.unpersist(blocking=false) 161 | prevG.unpersistVertices(blocking=false) 162 | prevG.edges.unpersist(blocking=false) 163 | // count the iteration 164 | i += 1 165 | } 166 | 167 | g 168 | } // end of apply 169 | 170 | } // end of class Pregel -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/exp/Profiler.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.exp 2 | 3 | import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM, Vector => BV} 4 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 5 | 6 | object Profiler { 7 | 8 | 9 | def main(args: Array[String]): Unit = { 10 | val map = new HyperXOpenHashMap[Int, Int]() 11 | (0 until 10).foreach(i => map.update(i, i *2)) 12 | 13 | println(map.iterator.map(i => i._1 + " " + i._2).reduce(_ + " ; " + _)) 14 | } 15 | 16 | } 17 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/FlatHyperedgePartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.Logging 4 | import org.apache.spark.hyperx._ 5 | import org.apache.spark.hyperx.util.HyperUtils 6 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 7 | 8 | import scala.reflect.ClassTag 9 | 10 | /** 11 | * Each hyperedge is dissembled into multiple (VertexId, HyperedgeId, SrcFlag) 12 | * tuples. The underlying structure is three big arrays. 13 | * 14 | * Additionally, it maintains a source index and a hyperedge index: 15 | * source index: source VertexId -> array of ids of all the relevant hyperedges 16 | * hyperedge index: hyperedge id -> the first index in the tuple arrays that 17 | * corresponds to the hyperedge 18 | * 19 | * note: merging data and hIndex will NOT reduce the memory consumption, because 20 | * Tuple2 has a significant overhead 21 | * 22 | */ 23 | private[hyperx] 24 | class FlatHyperedgePartition[ 25 | @specialized(Int, Long, Double, Byte, Char) ED: ClassTag, VD: ClassTag]( 26 | val vertexIds: Array[VertexId] = null, 27 | val hyperedgeIds: Array[HyperedgeId] = null, 28 | val srcFlags: Array[Boolean] = null, 29 | val data: HyperXOpenHashMap[HyperedgeId, ED] = null, 30 | val srcIndex: HyperXOpenHashMap[VertexId, Array[HyperedgeId]] = null, 31 | val hIndex: HyperXOpenHashMap[HyperedgeId, Int] = null, 32 | val vertices: VertexPartition[VD] = null, 33 | val activeSet: Option[VertexSet] = None 34 | ) extends Serializable with Logging { 35 | 36 | val size: Int = hIndex.size 37 | 38 | def withActiveSet(iter: Iterator[VertexId]) 39 | : FlatHyperedgePartition[ED, VD] = { 40 | val newActiveSet = new VertexSet() 41 | iter.foreach(newActiveSet.add) 42 | new FlatHyperedgePartition(vertexIds, hyperedgeIds, srcFlags, data, 43 | srcIndex, hIndex, vertices, Some(newActiveSet)) 44 | } 45 | 46 | def withActiveSet(set: Option[VertexSet]) 47 | : FlatHyperedgePartition[ED, VD] = { 48 | new FlatHyperedgePartition(vertexIds, hyperedgeIds, srcFlags, data, 49 | srcIndex, hIndex, vertices, set) 50 | } 51 | 52 | def withVertices[VD2: ClassTag](vertices_ : VertexPartition[VD2]) 53 | : FlatHyperedgePartition[ED, VD2] = { 54 | new FlatHyperedgePartition[ED, VD2](vertexIds, hyperedgeIds, srcFlags, data, 55 | srcIndex, hIndex, vertices_, activeSet) 56 | } 57 | 58 | def withData[ED2: ClassTag](data_ : HyperXOpenHashMap[HyperedgeId, ED2]) 59 | : FlatHyperedgePartition[ED2, VD] = { 60 | new FlatHyperedgePartition(vertexIds, hyperedgeIds, srcFlags, 61 | data_, srcIndex, hIndex, vertices, activeSet) 62 | } 63 | 64 | def updateVertices(iter: Iterator[(VertexId, VD)]) 65 | : FlatHyperedgePartition[ED, VD] = { 66 | this.withVertices[VD](vertices.innerJoinKeepLeft(iter)) 67 | } 68 | 69 | def isActive(vid: VertexId): Boolean ={ activeSet.get.contains(vid)} 70 | 71 | def isActive(vids: VertexSet): Boolean = { 72 | // vids.iterator.map(isActive).reduce(_ || _) 73 | val it = vids.iterator 74 | while (it.hasNext) { 75 | if (activeSet.get.contains(it.next())) { 76 | return true 77 | } 78 | } 79 | false 80 | } 81 | 82 | def numActives: Option[Int] = activeSet.map(_.size) 83 | 84 | def reverse: FlatHyperedgePartition[ED, VD] = { 85 | val flags = srcFlags.map(!_) 86 | new FlatHyperedgePartition(vertexIds, hyperedgeIds, flags, data, 87 | srcIndex, hIndex, vertices, activeSet) 88 | } 89 | 90 | def map[ED2: ClassTag](f: Hyperedge[ED] => ED2) 91 | : FlatHyperedgePartition[ED2, VD] = { 92 | val newData = new HyperXOpenHashMap[HyperedgeId, ED2](data.size) 93 | val hyperedge = new Hyperedge[ED]() 94 | val size = vertexIds.size 95 | var i = 0 96 | assert(size > 0) 97 | var currentId = hyperedgeIds(0) 98 | var srcSet, dstSet = new VertexSet() 99 | while(i < size) { 100 | if (currentId != hyperedgeIds(i)) { 101 | hyperedge.srcIds = srcSet 102 | hyperedge.dstIds = dstSet 103 | hyperedge.attr = data(currentId) 104 | newData(currentId) = f(hyperedge) 105 | srcSet = new VertexSet() 106 | dstSet = new VertexSet() 107 | } 108 | currentId = hyperedgeIds(i) 109 | if (srcFlags(i)) { 110 | srcSet.add(vertexIds(i)) 111 | } else { 112 | dstSet.add(vertexIds(i)) 113 | } 114 | i += 1 115 | } 116 | // don't forget the last one 117 | if (srcSet.size > 0 && dstSet.size > 0) { 118 | hyperedge.srcIds = srcSet 119 | hyperedge.dstIds = dstSet 120 | hyperedge.attr = data(currentId) 121 | newData(currentId) = f(hyperedge) 122 | } 123 | this.withData(newData) 124 | } 125 | 126 | def map[ED2: ClassTag](iter: Iterator[(HyperedgeId, ED2)]) 127 | : FlatHyperedgePartition[ED2, VD] = { 128 | val seq = iter.toIndexedSeq 129 | val newData = new HyperXOpenHashMap[HyperedgeId, ED2]() 130 | seq.foreach{i => 131 | // assert(data.hasKey(i._1)) 132 | newData(i._1) = i._2 133 | } 134 | // assert(newData.size == data.size) 135 | this.withData(newData) 136 | } 137 | 138 | def filter(hpred: HyperedgeTuple[VD, ED] => Boolean, 139 | vpred: HyperAttr[VD] => Boolean) 140 | : FlatHyperedgePartition[ED, VD] = { 141 | val filtered = tupleIterator().filter(t => 142 | vpred(t.srcAttr) && vpred(t.dstAttr) && hpred(t) 143 | ) 144 | val builder = new FlatHyperedgePartitionBuilder[ED, VD]() 145 | filtered.foreach{h => 146 | builder.add(h.srcAttr.keySet, h.dstAttr.keySet, 147 | h.id, h.attr) 148 | } 149 | builder.toFlatHyperedgePartition.withVertices(vertices) 150 | .withActiveSet(activeSet) 151 | } 152 | 153 | def tupleIterator(includeSrc: Boolean = true, includeDst: Boolean = true) 154 | : Iterator[HyperedgeTuple[VD, ED]] = { 155 | new FlatHyperedgeTupleIterator(this, includeSrc, includeDst) 156 | } 157 | 158 | def foreach(f: Hyperedge[ED] => Unit): Unit = { 159 | iterator.foreach(f) 160 | } 161 | 162 | def iterator = new Iterator[Hyperedge[ED]]{ 163 | private[this] val hyperedge = new Hyperedge[ED]() 164 | private[this] var index = 0 165 | private[this] var lastPos = 0 166 | 167 | override def hasNext: Boolean = index < FlatHyperedgePartition.this.size 168 | 169 | // the hyperedgeId is not exposed as it doesn't matter to the outer scope 170 | override def next(): Hyperedge[ED] = { 171 | hyperedge.srcIds = new VertexSet() 172 | hyperedge.dstIds = new VertexSet() 173 | val pos = hIndex.nextPos(lastPos) 174 | lastPos = pos + 1 175 | var i = hIndex._values(pos) 176 | val currentId = hyperedgeIds(i) 177 | while(i < hyperedgeIds.size && currentId == hyperedgeIds(i)) { 178 | val vid = vertexIds(i) 179 | if (srcFlags(i)) hyperedge.srcIds.add(vid) 180 | else hyperedge.dstIds.add(vid) 181 | i += 1 182 | } 183 | hyperedge.attr = data(currentId) 184 | hyperedge.id = currentId 185 | index += 1 186 | hyperedge 187 | } 188 | } 189 | 190 | // todo: srcIds are not sorted, grouping involves a quadratic cost 191 | def groupHyperedges(merge: (ED, ED) => ED) 192 | : FlatHyperedgePartition[ED, VD] = { 193 | val builder = new FlatHyperedgePartitionBuilder[ED, VD]() 194 | val merged = new Array[(VertexSet, VertexSet)](size) 195 | var currSrcIds: VertexSet = null.asInstanceOf[VertexSet] 196 | var currDstIds: VertexSet = null.asInstanceOf[VertexSet] 197 | var currAttr: ED = null.asInstanceOf[ED] 198 | val outerIter = iterator 199 | var i = 0 200 | while(outerIter.hasNext) { 201 | val outerH = outerIter.next() 202 | currSrcIds = outerH.srcIds 203 | currDstIds = outerH.dstIds 204 | if (merged.count(r => HyperUtils.is(r._1, currSrcIds) && 205 | HyperUtils.is(r._2, currDstIds)) == 0) { 206 | currAttr = outerH.attr 207 | // val outerPos = outerIter.pos 208 | val innerIter = iterator 209 | // innerIter.pos = outerPos + 1 210 | while(innerIter.hasNext) { 211 | val inner = innerIter.next() 212 | if (HyperUtils.is(inner.srcIds, currSrcIds) && 213 | HyperUtils.is(inner.dstIds, currDstIds)) { 214 | currAttr = merge(currAttr, inner.attr) 215 | } 216 | } 217 | builder.add(currSrcIds, currDstIds, i, currAttr) 218 | i += 1 219 | } 220 | } 221 | builder.toFlatHyperedgePartition.withVertices(vertices) 222 | .withActiveSet(activeSet) 223 | } 224 | 225 | def innerJoin[ED2: ClassTag, ED3: ClassTag](other: FlatHyperedgePartition[ED2, _]) 226 | (f: (VertexSet, VertexSet, ED, ED2) => ED3): FlatHyperedgePartition[ED3, VD] = { 227 | val builder = new FlatHyperedgePartitionBuilder[ED3, VD]() 228 | val thisIter = this.iterator 229 | var i = 0 230 | while(thisIter.hasNext) { 231 | val thisH = thisIter.next() 232 | val otherIter = other.iterator 233 | while(otherIter.hasNext) { 234 | val otherH = otherIter.next() 235 | if (HyperUtils.is(thisH.srcIds, otherH.srcIds) && 236 | HyperUtils.is(thisH.dstIds, otherH.dstIds)) { 237 | builder.add(thisH.srcIds, thisH.dstIds, i, 238 | f(thisH.srcIds, thisH.dstIds, thisH.attr, otherH.attr)) 239 | } 240 | } 241 | i += 1 242 | } 243 | builder.toFlatHyperedgePartition.withVertices(vertices) 244 | .withActiveSet(activeSet) 245 | } 246 | 247 | def sourceSize: Int = srcIndex.size 248 | 249 | def upgradeIterator(hyperedgeIterator: Iterator[Hyperedge[ED]], 250 | includeSrc: Boolean = true, includeDst: Boolean = true) 251 | : Iterator[HyperedgeTuple[VD, ED]] = { 252 | new ReusingFlatHyperedgeTupleIterator(hyperedgeIterator, this, 253 | includeSrc, includeDst) 254 | } 255 | 256 | def indexIterator(srcIdPred: VertexId => Boolean): Iterator[Hyperedge[ED]] = { 257 | srcIndex.iterator.filter(v => srcIdPred(v._1)).flatMap(each => arrayIndexIterator(each._1)) 258 | } 259 | 260 | private def arrayIndexIterator(srcId: VertexId) = new Iterator[Hyperedge[ED]] { 261 | private[this] val hyperedge = new Hyperedge[ED]() 262 | private[this] var pos = 0 263 | 264 | override def hasNext: Boolean = { 265 | pos >= 0 && pos < srcIndex(srcId).size 266 | } 267 | 268 | override def next(): Hyperedge[ED] = { 269 | val hyperedgeId = srcIndex(srcId)(pos) 270 | var i = hIndex(hyperedgeId) 271 | hyperedge.srcIds = new VertexSet 272 | hyperedge.dstIds = new VertexSet 273 | while(i < hyperedgeIds.size && hyperedgeId == hyperedgeIds(i)) { 274 | if (srcFlags(i)) hyperedge.srcIds.add(vertexIds(i)) 275 | else hyperedge.dstIds.add(vertexIds(i)) 276 | i += 1 277 | } 278 | hyperedge.attr = data(hyperedgeId) 279 | pos += 1 280 | hyperedge 281 | } 282 | } 283 | } 284 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/FlatHyperedgePartitionBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.hyperx.{VertexSet, VertexId, HyperedgeId, Hyperedge} 4 | import org.apache.spark.hyperx.util.collection.{HyperXOpenHashMap, 5 | HyperXPrimitiveVector} 6 | 7 | import scala.reflect.ClassTag 8 | 9 | //private[hyperx] 10 | class FlatHyperedgePartitionBuilder[ 11 | @specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( 12 | size: Int = 64, withIndex: Boolean = true) extends Serializable { 13 | 14 | var hyperedges = new HyperXPrimitiveVector[Hyperedge[ED]](size) 15 | var hyperedgeIds = new HyperXPrimitiveVector[HyperedgeId](size) 16 | var hIndices = new HyperXOpenHashMap[HyperedgeId, Int]() 17 | var srcIndices = 18 | new HyperXOpenHashMap[VertexId, HyperXPrimitiveVector[HyperedgeId]]() 19 | var dstIndices = 20 | new HyperXOpenHashMap[VertexId, HyperXPrimitiveVector[HyperedgeId]]() 21 | 22 | def add(src: VertexSet, dst: VertexSet, hid: HyperedgeId, data: ED) 23 | : Unit = { 24 | hyperedges += Hyperedge(src, dst, data) 25 | hyperedgeIds += hid 26 | } 27 | 28 | def toFlatHyperedgePartition: FlatHyperedgePartition[ED, VD] = { 29 | val hyperedgeArray = hyperedges.trim().array 30 | val hyperedgeIdArray = hyperedgeIds.trim().array 31 | val vertexIds = new HyperXPrimitiveVector[VertexId]() 32 | val hIds = new HyperXPrimitiveVector[HyperedgeId]() 33 | val srcFlags = new HyperXPrimitiveVector[Boolean]() 34 | val data = new HyperXOpenHashMap[HyperedgeId, ED]() 35 | val allVertices = new VertexSet() 36 | 37 | if (hyperedgeArray.size > 0) { 38 | var i = 0 39 | while(i < hyperedgeArray.size) { 40 | val h = hyperedgeArray(i) 41 | val hId = hyperedgeIdArray(i) 42 | val pos = vertexIds.size 43 | hIndices.update(hId, pos) 44 | h.srcIds.iterator.foreach{v => 45 | vertexIds += v 46 | hIds += hId 47 | srcFlags += true 48 | updateIndex(srcIndices, v, hId) 49 | allVertices.add(v) 50 | } 51 | h.dstIds.iterator.foreach{v => 52 | vertexIds += v 53 | hIds += hId 54 | srcFlags += false 55 | updateIndex(dstIndices, v, hId) 56 | allVertices.add(v) 57 | } 58 | data(hId) = hyperedgeArray(i).attr 59 | i += 1 60 | } 61 | } 62 | 63 | val builtIndex = if (!withIndex) null else makeIndex 64 | 65 | // this was a notorious bug that sucked more than 2 hours to hunt down, 66 | // the mistake is using allVertices.size instead of allVertices.capacity, 67 | // which leads the values in the vertex partition to be of length much 68 | // smaller than it should be! 69 | val vertices = new VertexPartition( 70 | allVertices, new Array[VD](allVertices.capacity), allVertices.getBitSet 71 | ) 72 | 73 | new FlatHyperedgePartition(vertexIds.trim().array, 74 | hIds.trim().array, srcFlags.trim().array, data, 75 | builtIndex, hIndices, vertices) 76 | 77 | } 78 | 79 | private def updateIndex( 80 | index: HyperXOpenHashMap[VertexId, HyperXPrimitiveVector[HyperedgeId]], 81 | vid: VertexId, hId: HyperedgeId) = { 82 | if (!index.hasKey(vid)) { 83 | index.update(vid, new HyperXPrimitiveVector[HyperedgeId]()) 84 | } 85 | index(vid) += hId 86 | } 87 | 88 | private def makeIndex = { 89 | val map = new HyperXOpenHashMap[VertexId, Array[HyperedgeId]]() 90 | srcIndices.foreach(m => map.update(m._1, m._2.trim().array)) 91 | map 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/FlatHyperedgeTupleIterator.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.Logging 4 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 5 | import org.apache.spark.hyperx.{Hyperedge, HyperedgeTuple, VertexId} 6 | 7 | import scala.reflect.ClassTag 8 | 9 | private[impl] 10 | class FlatHyperedgeTupleIterator[VD: ClassTag, ED: ClassTag]( 11 | val hyperedgePartition: FlatHyperedgePartition[ED, VD], 12 | val includeSrc: Boolean, val includeDst: Boolean) 13 | extends Iterator[HyperedgeTuple[VD, ED]] with Logging{ 14 | 15 | private var index = 0 16 | private var lastPos = 0 17 | 18 | override def hasNext: Boolean = { 19 | index < hyperedgePartition.size 20 | } 21 | 22 | override def next() = { 23 | val tuple = new HyperedgeTuple[VD, ED] 24 | tuple.srcAttr = new HyperXOpenHashMap[VertexId, VD]() 25 | tuple.dstAttr = new HyperXOpenHashMap[VertexId, VD]() 26 | val pos = hyperedgePartition.hIndex.nextPos(lastPos) 27 | lastPos = pos + 1 28 | var i = hyperedgePartition.hIndex._values(pos) 29 | val currentId = hyperedgePartition.hyperedgeIds(i) 30 | while(i < hyperedgePartition.hyperedgeIds.size && 31 | currentId == hyperedgePartition.hyperedgeIds(i)) { 32 | val vid = hyperedgePartition.vertexIds(i) 33 | if (hyperedgePartition.srcFlags(i)) { 34 | if (includeSrc) { 35 | tuple.srcAttr.update(vid, hyperedgePartition.vertices(vid)) 36 | } else { 37 | tuple.srcAttr.update(vid, null.asInstanceOf[VD]) 38 | } 39 | } else { 40 | if (includeDst) { 41 | tuple.dstAttr.update(vid, hyperedgePartition.vertices(vid)) 42 | } else { 43 | tuple.dstAttr.update(vid, null.asInstanceOf[VD]) 44 | } 45 | } 46 | i += 1 47 | } 48 | tuple.id = currentId 49 | tuple.attr = hyperedgePartition.data(currentId) 50 | index += 1 51 | tuple 52 | } 53 | } 54 | 55 | 56 | private[impl] 57 | class ReusingFlatHyperedgeTupleIterator[VD: ClassTag, ED: ClassTag]( 58 | val hyperedgeIter: Iterator[Hyperedge[ED]], 59 | val hyperedgePartition: FlatHyperedgePartition[ED, VD], 60 | val includeSrc: Boolean, val includeDst: Boolean) 61 | extends Iterator[HyperedgeTuple[VD, ED]] with Logging{ 62 | 63 | private val tuple = new HyperedgeTuple[VD, ED] 64 | 65 | override def hasNext: Boolean = hyperedgeIter.hasNext 66 | 67 | override def next() = { 68 | tuple.set(hyperedgeIter.next()) 69 | if (includeSrc) { 70 | tuple.srcAttr.foreach(pair => tuple.srcAttr.update(pair._1, 71 | hyperedgePartition.vertices(pair._1))) 72 | } 73 | if (includeDst) { 74 | tuple.dstAttr.foreach(pair => tuple.dstAttr.update(pair._1, 75 | hyperedgePartition.vertices(pair._1))) 76 | } 77 | tuple 78 | } 79 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/HyperedgePartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.Accumulator 4 | import org.apache.spark.hyperx._ 5 | import org.apache.spark.hyperx.util.HyperUtils 6 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 7 | 8 | import scala.reflect.{ClassTag, classTag} 9 | 10 | /** 11 | * A collection of hyperedges stored in columnar format, 12 | * along with any vertex attributes referenced. 13 | * The hyperedges are stored in 3 large columnar arrays (src, dst, 14 | * attribute). There is an optional 15 | * active vertex set for filtering the computation on hyperedges. 16 | * 17 | * Forked from GraphX 2.10, modified by Jin Huang 18 | */ 19 | private[hyperx] 20 | class HyperedgePartition[ 21 | @specialized(Char, Int, Boolean, Byte, Long, Float, Double) 22 | ED: ClassTag,VD: ClassTag]( 23 | val srcIds: Array[VertexSet] = null, 24 | val dstIds: Array[VertexSet] = null, 25 | val data: Array[ED] = null, 26 | val srcIndex: HyperXOpenHashMap[VertexId, Array[Int]] = null, 27 | val vertices: VertexPartition[VD] = null, 28 | val activeSet: Option[VertexSet] = None 29 | ) extends Serializable { 30 | 31 | /** The number of hyperedges in this partition. */ 32 | val size: Int = srcIds.size 33 | 34 | /** Return a new `HyperedgePartition` with the specified active set, 35 | * provided as an iterator. */ 36 | def withActiveSet(iter: Iterator[VertexId]): HyperedgePartition[ED, VD] = { 37 | val activeArray = iter.toArray 38 | val newActiveSet: VertexSet = new VertexSet() 39 | activeArray.foreach(newActiveSet.add) 40 | new HyperedgePartition(srcIds, dstIds, data, srcIndex, vertices, 41 | Some(newActiveSet)) 42 | } 43 | 44 | /** Return a new `HyperedgePartition` with the specified active set. */ 45 | def withActiveSet(activeSet_ : Option[VertexSet]): HyperedgePartition[ED, 46 | VD] = { 47 | new HyperedgePartition(srcIds, dstIds, data, srcIndex, vertices, 48 | activeSet_) 49 | } 50 | 51 | /** Return a new `HyperedgePartition` with updates to vertex attributes 52 | * specified in `iter`. */ 53 | def updateVertices(iter: Iterator[(VertexId,VD)]): HyperedgePartition[ED, VD] = { 54 | this.withVertices(vertices.innerJoinKeepLeft(iter)) 55 | } 56 | 57 | /** Look up vid in activeSet, throwing an exception if it is None. */ 58 | def isActive(vid: VertexId): Boolean = { 59 | activeSet.get.contains(vid) 60 | } 61 | 62 | /** Look up vids in activeSet, throwing an exception if it is None. */ 63 | def isActive(vids: VertexSet): Boolean = { 64 | val it = vids.iterator 65 | while (it.hasNext) { 66 | if (activeSet.get.contains(it.next())) { 67 | return true 68 | } 69 | } 70 | false 71 | } 72 | 73 | /** The number of active vertices, if any exists */ 74 | def numActives: Option[Int] = activeSet.map(_.size) 75 | 76 | /** 77 | * Reverse all the hyperedges in this partition 78 | * @return a new hyperedge partition with all the hyperedges reversed 79 | */ 80 | def reverse: HyperedgePartition[ED, VD] = { 81 | val builder = new HyperedgePartitionBuilder(size)(classTag[ED], 82 | classTag[VD]) 83 | for (h <- iterator) { 84 | builder.add(h.dstIds, h.srcIds, h.attr) 85 | } 86 | builder.toHyperedgePartition.withVertices(vertices) 87 | .withActiveSet(activeSet) 88 | } 89 | 90 | /** 91 | * Construct a new hyperedge partition by applying the function f to all 92 | * hyperedges 93 | * in this partition 94 | * 95 | * @param f a function from an hyperedge to a new attribute 96 | * @tparam ED2 the type of the new attribute 97 | * @return a new hyperedge partition with the result of the function `f` 98 | * applied to each edge 99 | */ 100 | def map[ED2: ClassTag](f: Hyperedge[ED] => ED2): HyperedgePartition[ED2, 101 | VD] = { 102 | val newData = new Array[ED2](data.size) 103 | val hyperedge = new Hyperedge[ED]() 104 | val size = data.size 105 | var i = 0 106 | while (i < size) { 107 | hyperedge.srcIds = srcIds(i) 108 | hyperedge.dstIds = dstIds(i) 109 | hyperedge.attr = data(i) 110 | newData(i) = f(hyperedge) 111 | i += 1 112 | } 113 | this.withData(newData) 114 | } 115 | 116 | /** 117 | * Construct a new hyperedge partition by using the edge attribute 118 | * contained in the iterator 119 | * 120 | * @note The input iterator should return hyperedge attributes in the 121 | * order of 122 | * the hyperedges returned by `HyperedgePartition.iterator` and 123 | * should return 124 | * attributes equal to the number of hyperedges 125 | * 126 | * @param iter an iterator for the new attribute values 127 | * @tparam ED2 the type of new attributes 128 | * @return a new hyperedge partition with the attribute values replaced 129 | */ 130 | def map[ED2: ClassTag](iter: Iterator[ED2]): HyperedgePartition[ED2, VD] = { 131 | val newData = new Array[ED2](data.size) 132 | var i = 0 133 | while (iter.hasNext) { 134 | newData(i) = iter.next() 135 | i += 1 136 | } 137 | assert(newData.size == i) 138 | this.withData(newData) 139 | } 140 | 141 | /** Return a new `HyperedgePartition` with the specified hyperedge data. */ 142 | def withData[ED2: ClassTag](data_ : Array[ED2]): HyperedgePartition[ED2, 143 | VD] = { 144 | new HyperedgePartition(srcIds, dstIds, data_, srcIndex, vertices, 145 | activeSet) 146 | } 147 | 148 | /** 149 | * Constrcut a new hyperedge partition containing only the hyperedge 150 | * matching `epred` 151 | * and where both vertex sets match `vpred` 152 | */ 153 | def filter( 154 | epred: HyperedgeTuple[VD, ED] => Boolean, 155 | vpred: HyperAttr[VD] => Boolean): 156 | HyperedgePartition[ED, VD] = { 157 | val filtered = tupleIterator().filter(et => 158 | vpred(et.srcAttr) && vpred(et.dstAttr) && epred(et)) 159 | val builder = new HyperedgePartitionBuilder[ED, VD] 160 | for (e <- filtered) { 161 | builder.add(e.srcAttr.keySet, e.dstAttr.keySet, e.attr) 162 | } 163 | builder.toHyperedgePartition.withVertices(vertices) 164 | .withActiveSet(activeSet) 165 | } 166 | 167 | /** 168 | * Get an iterator over the hyperedge tuple in this partition 169 | * 170 | * It is safe to keep references to the objects from this iterator 171 | */ 172 | def tupleIterator(includeSrc: Boolean = true, includeDst: Boolean = true) 173 | : Iterator[HyperedgeTuple[VD, ED]] = 174 | new HyperedgeTupleIterator(this, includeSrc, includeDst) 175 | 176 | /** 177 | * Apply the function `f` to all hyperedges in this partition 178 | */ 179 | def foreach(f: Hyperedge[ED] => Unit): Unit = { 180 | iterator.foreach(f) 181 | } 182 | 183 | /** 184 | * Get an iterator over the hyperedges in this partition 185 | * 186 | * Be careful not to keep references to the objects from this iterator. 187 | * 188 | * @return an iterator over hyperedges in the partition 189 | */ 190 | def iterator = new Iterator[Hyperedge[ED]] { 191 | private[this] val hyperedge = new Hyperedge[ED]() 192 | private[this] var pos = 0 193 | 194 | override def hasNext: Boolean = pos < HyperedgePartition.this.size 195 | 196 | override def next(): Hyperedge[ED] = { 197 | hyperedge.srcIds = srcIds(pos) 198 | hyperedge.dstIds = dstIds(pos) 199 | hyperedge.attr = data(pos) 200 | pos += 1 201 | hyperedge 202 | } 203 | } 204 | 205 | /** 206 | * Merge all the hyperedges with the same src and dest set into a single 207 | * hyperedge 208 | * using the `merge` function 209 | */ 210 | def groupHyperedges(merge: (ED, ED) => ED): HyperedgePartition[ED, VD] = { 211 | val builder = new HyperedgePartitionBuilder[ED, VD] 212 | val currSrcIds: VertexSet = null.asInstanceOf[VertexSet] 213 | val currDstIds: VertexSet = null.asInstanceOf[VertexSet] 214 | var currAttr: ED = null.asInstanceOf[ED] 215 | var i = 0 216 | while (i < size) { 217 | if (i > 0 && HyperUtils.is(currSrcIds, srcIds(i)) && HyperUtils 218 | .is(currDstIds, dstIds(i))) { 219 | currAttr = merge(currAttr, data(i)) 220 | } 221 | else { 222 | if (i > 0) { 223 | builder.add(currSrcIds, currDstIds, currAttr) 224 | } 225 | } 226 | i += 1 227 | } 228 | if (size > 0) { 229 | builder.add(currSrcIds, currDstIds, currAttr) 230 | } 231 | builder.toHyperedgePartition.withVertices(vertices) 232 | .withActiveSet(activeSet) 233 | } 234 | 235 | /** 236 | * Apply `f` to all hyperedges present in both `this` and `other` and 237 | * return new `HyperedgePartition` 238 | * containing the resulting hyperedges 239 | * 240 | * This is rather costly, as we don't have any clustered index on neither 241 | * partition. 242 | * The join is carried out via a naive cartesian product 243 | * @todo To implement a more efficient join algorithm 244 | **/ 245 | def innerJoin[ED2: ClassTag, ED3: ClassTag] 246 | (other: HyperedgePartition[ED2, _]) 247 | (f: (VertexSet, VertexSet, ED, ED2) => ED3): HyperedgePartition[ED3, VD] = { 248 | val builder = new HyperedgePartitionBuilder[ED3, VD] 249 | var i = 0 250 | var j = 0 251 | while (i < this.size) { 252 | while (j < this.size) { 253 | if (HyperUtils.is(srcIds(i), other.srcIds(j)) && HyperUtils 254 | .is(dstIds(i), other.dstIds(j))) { 255 | builder.add(srcIds(i), dstIds(i), f(srcIds(i), dstIds(i), 256 | this.data(i), other.data(j))) 257 | } 258 | j += 1 259 | } 260 | i += 1 261 | } 262 | builder.toHyperedgePartition.withVertices(vertices) 263 | .withActiveSet(activeSet) 264 | } 265 | 266 | /** Return a new `HyperedgePartition` with the specified vertex partition 267 | * . */ 268 | def withVertices[VD2: ClassTag](vertices_ : VertexPartition[VD2]): 269 | HyperedgePartition[ED, VD2] = { 270 | new HyperedgePartition(srcIds, dstIds, data, srcIndex, vertices_, 271 | activeSet) 272 | } 273 | 274 | /** The number of unique source vertices in the partition. */ 275 | def indexSize: Int = srcIndex.size 276 | 277 | /** 278 | * Upgrade the given hyperedge iterator into a tuple iterator 279 | * 280 | * Be careful not to keep references to the objects from this iterator. 281 | */ 282 | def upgradeIterator( 283 | hyperedgeIter: Iterator[Hyperedge[ED]], 284 | includeSrc: Boolean = true, 285 | includeDst: Boolean = true) 286 | : Iterator[HyperedgeTuple[VD, ED]] = { 287 | new ReusingHyperedgeTupleIterator(hyperedgeIter, this, includeSrc, includeDst) 288 | } 289 | 290 | def upgradeIteratorP( 291 | hyperedgeIter: Iterator[Hyperedge[ED]],tracker: Accumulator[Int], 292 | includeSrc: Boolean = true, 293 | includeDst: Boolean = true) 294 | : Iterator[HyperedgeTuple[VD, ED]] = { 295 | val start = System.currentTimeMillis() 296 | val ret = new ReusingHyperedgeTupleIterator(hyperedgeIter, this, includeSrc, includeDst) 297 | tracker += (System.currentTimeMillis() - start).toInt 298 | ret 299 | } 300 | 301 | /** 302 | * Get an iterator over the hyperedges in this partition whose source vertex id match 303 | * srcIdsPred. The iterator is generated using an index scan, so it is efficient at skipping 304 | * edges that don't match srcIdsPred. 305 | * 306 | * Be careful not to keep references to the objects from this iterator. 307 | */ 308 | def indexIterator(srcIdPred: VertexId => Boolean): Iterator[Hyperedge[ED]] = { 309 | srcIndex.iterator.filter(kv => srcIdPred(kv._1)).flatMap(each => arrayIndexIterator(each._1)) 310 | } 311 | 312 | /** 313 | * Get an iterator over the array index of hyperedges in this partition with source vertex 314 | * id `srcId` 315 | * 316 | * Be careful not to keep references to the objects from this iterator. 317 | * @param srcId the source vertex id 318 | * @return an iterator over the hyperedges that has source vertex `srcId` 319 | */ 320 | private def arrayIndexIterator(srcId: VertexId) = new Iterator[Hyperedge[ED]] { 321 | private[this] val hyperedge = new Hyperedge[ED]() 322 | private[this] var pos = 0 323 | 324 | override def hasNext: Boolean = { 325 | pos >= 0 && pos < HyperedgePartition.this.srcIndex(srcId).size 326 | } 327 | 328 | override def next(): Hyperedge[ED] = { 329 | assert(srcIds(srcIndex(srcId)(pos)).contains(srcId)) 330 | hyperedge.srcIds = srcIds(srcIndex(srcId)(pos)) 331 | hyperedge.dstIds = dstIds(srcIndex(srcId)(pos)) 332 | hyperedge.attr = data(srcIndex(srcId)(pos)) 333 | pos += 1 334 | hyperedge 335 | } 336 | } 337 | } 338 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/HyperedgePartitionBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.hyperx.util.collection.{HyperXOpenHashMap, HyperXPrimitiveVector} 4 | import org.apache.spark.hyperx.{Hyperedge, VertexId, VertexSet} 5 | 6 | import scala.reflect.ClassTag 7 | 8 | /** Forked from GraphX 2.10, modified by Jin Huang */ 9 | private[hyperx] 10 | class HyperedgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, 11 | VD: ClassTag](size: Int = 64, withIndex: Boolean = true) extends 12 | Serializable { 13 | var hyperedges = new HyperXPrimitiveVector[Hyperedge[ED]](size) 14 | var indices = new HyperXOpenHashMap[VertexId, HyperXPrimitiveVector[Int]] 15 | var dstIndices = 16 | new HyperXOpenHashMap[VertexId, HyperXPrimitiveVector[Int]]() 17 | 18 | /** Add a new hyperedge to the partition. */ 19 | def add(src: VertexSet, dst: VertexSet, d: ED): Unit = { 20 | hyperedges += Hyperedge(src, dst, d) 21 | } 22 | 23 | def toHyperedgePartition: HyperedgePartition[ED, VD] = { 24 | val hyperedgeArray = hyperedges.trim().array 25 | val srcIds = new Array[VertexSet](hyperedgeArray.size) 26 | val dstIds = new Array[VertexSet](hyperedgeArray.size) 27 | val data = new Array[ED](hyperedgeArray.size) 28 | 29 | if (hyperedgeArray.length > 0) { 30 | var i = 0 31 | while (i < hyperedges.size) { 32 | srcIds(i) = hyperedgeArray(i).srcIds 33 | dstIds(i) = hyperedgeArray(i).dstIds 34 | data(i) = hyperedgeArray(i).attr 35 | encounterVertices(srcIds(i), i) 36 | encounterDstVertices(dstIds(i), i) 37 | i += 1 38 | } 39 | } 40 | 41 | val builtIndex = if (!withIndex) null else makeIndex 42 | // populate the vertex partition 43 | val vertexIds = indices.keySet 44 | dstIndices.keySet.iterator.foreach(vertexIds.add) 45 | 46 | val vertices = new VertexPartition( 47 | vertexIds, new Array[VD](vertexIds.capacity), vertexIds.getBitSet) 48 | 49 | val partition = 50 | new HyperedgePartition(srcIds, dstIds, data, builtIndex, vertices) 51 | partition 52 | } 53 | 54 | private def encounterVertices(vids: VertexSet, i: Int) = { 55 | val it = vids.iterator 56 | while (it.hasNext) { 57 | val curId = it.next() 58 | if (indices.apply(curId) == null) { 59 | if (withIndex) 60 | indices.update(curId, new HyperXPrimitiveVector[Int]()) 61 | else 62 | indices.update(curId, null) 63 | } 64 | if (withIndex) 65 | indices.apply(curId) += i 66 | } 67 | } 68 | 69 | private def encounterDstVertices(vids: VertexSet, i: Int) = { 70 | val it = vids.iterator 71 | while(it.hasNext) { 72 | val curId = it.next() 73 | dstIndices.update(curId, null) 74 | } 75 | } 76 | 77 | 78 | private def makeIndex: HyperXOpenHashMap[VertexId, Array[Int]] = { 79 | val it = indices.iterator 80 | val map = new HyperXOpenHashMap[VertexId, Array[Int]]() 81 | while (it.hasNext) { 82 | val cur = it.next() 83 | map.update(cur._1, cur._2.trim().array) 84 | } 85 | map 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/HyperedgeTupleIterator.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.hyperx.util.HyperUtils 4 | import org.apache.spark.hyperx.{Hyperedge, HyperedgeTuple} 5 | 6 | import scala.reflect.ClassTag 7 | 8 | /** 9 | * The Iterator type returned when constructing hyperedge tuples. 10 | */ 11 | private[impl] 12 | class HyperedgeTupleIterator[VD: ClassTag, ED: ClassTag]( 13 | val hyperedgePartition: HyperedgePartition[ED, VD], val includeSrc: Boolean, 14 | val includeDst: Boolean) 15 | extends Iterator[HyperedgeTuple[VD, ED]] { 16 | 17 | private var pos = 0 18 | 19 | override def hasNext: Boolean = pos < hyperedgePartition.size 20 | 21 | override def next() = { 22 | val tuple = new HyperedgeTuple[VD, ED] 23 | tuple.srcAttr = HyperUtils.init(hyperedgePartition.srcIds(pos)) 24 | if (includeSrc) { 25 | for (it <- tuple.srcAttr.iterator) { 26 | tuple.srcAttr.update(it._1, hyperedgePartition.vertices(it._1)) 27 | } 28 | } 29 | tuple.dstAttr = HyperUtils.init(hyperedgePartition.dstIds(pos)) 30 | if (includeDst) { 31 | for (it <- tuple.dstAttr.iterator) { 32 | tuple.dstAttr.update(it._1, hyperedgePartition.vertices(it._1)) 33 | } 34 | } 35 | tuple.attr = hyperedgePartition.data(pos) 36 | pos += 1 37 | tuple 38 | } 39 | } 40 | 41 | /** 42 | * An Iterator type for internal use that reuses HyperedgeTuple objects 43 | */ 44 | private[impl] 45 | class ReusingHyperedgeTupleIterator[VD: ClassTag, ED: ClassTag]( 46 | val hyperedgeIter: Iterator[Hyperedge[ED]], 47 | val hyperedgePartition: HyperedgePartition[ED, VD], 48 | val includeSrc: Boolean, val includeDst: Boolean) extends 49 | Iterator[HyperedgeTuple[VD, ED]] { 50 | 51 | private val tuple = new HyperedgeTuple[VD, ED] 52 | 53 | override def hasNext = hyperedgeIter.hasNext 54 | 55 | override def next() = { 56 | tuple.set(hyperedgeIter.next()) 57 | if (includeSrc) { 58 | for (it <- tuple.srcAttr.iterator) { 59 | tuple.srcAttr.update(it._1, hyperedgePartition.vertices(it._1)) 60 | } 61 | } 62 | if (includeDst) { 63 | for (it <- tuple.dstAttr.iterator) { 64 | tuple.dstAttr.update(it._1, hyperedgePartition.vertices(it._1)) 65 | } 66 | } 67 | tuple 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/ReplicatedVertexView.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.impl 2 | 3 | import org.apache.spark.{Accumulator, Logging} 4 | import org.apache.spark.SparkContext._ 5 | import org.apache.spark.hyperx.{HyperedgeRDD, VertexId, VertexRDD} 6 | import org.apache.spark.rdd.RDD 7 | 8 | import scala.reflect.ClassTag 9 | 10 | /** 11 | * Manages shipping vertex attributes to the hyperedge partitions of an 12 | * [[org.apache.spark.hyperx.HyperedgeRDD]]. Vertex attributes may be 13 | * partially shipped to 14 | * construct a tuple view with vertex attributes on only one side, 15 | * and they may be updated. 16 | * An active vertex set may additionally be shipped to the hyperedge 17 | * partitions. Be careful 18 | * not to store a reference to `hyperedges`, since it may be modified when 19 | * the attribute 20 | * shipping level is upgraded. 21 | * 22 | * Forked from GraphX 2.10, modified by Jin Huang 23 | */ 24 | private[hyperx] 25 | class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( 26 | var hyperedges: HyperedgeRDD[ED, VD], var hasSrcIds: Boolean = false, 27 | var hasDstIds: Boolean = false) extends Logging with Serializable { 28 | 29 | /** 30 | * Return a new `ReplicatedVertexView` with the specified `HyperedgeRDD`, 31 | * which must have 32 | * the same shipping level 33 | */ 34 | def withHyperedges[VD2: ClassTag, ED2: ClassTag]( 35 | hyperedges_ : HyperedgeRDD[ED2, VD2]): 36 | ReplicatedVertexView[VD2, ED2] = { 37 | new ReplicatedVertexView(hyperedges_, hasSrcIds, hasDstIds) 38 | } 39 | 40 | /** 41 | * Return a new `ReplicatedVertexView` where hyperedges are reversed and 42 | * shipping levels 43 | * are swapped to match. 44 | * @return 45 | */ 46 | def reverse() = { 47 | val newHyperedges = hyperedges.mapHyperedgePartitions((pid, part) => 48 | part.reverse) 49 | new ReplicatedVertexView(newHyperedges, hasDstIds, hasSrcIds) 50 | } 51 | 52 | /** 53 | * Upgrade the shipping level in-place to the specified levels by 54 | * shipping vertex attributes from 55 | * `vertices`. This operation modifies the `ReplicatedVertexView`, 56 | * and callers can access 57 | * `hyperedges` afterwards to obtain the upgraded view. 58 | */ 59 | def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, 60 | includeDst: Boolean) 61 | : Unit = { 62 | val shipSrc = includeSrc && !hasSrcIds 63 | val shipDst = includeDst && !hasDstIds 64 | if (shipSrc || shipDst) { 65 | val shippedVerts: RDD[(Int, VertexAttributeBlock[VD])] = 66 | vertices.shipVertexAttributes(shipSrc, shipDst).setName( 67 | ("ReplicatedVertexView.upgrade(%s, %s) - " + 68 | "shippedVerts %s %s (broadcast)") 69 | .format(includeSrc, includeDst, shipSrc, shipDst)) 70 | .partitionBy(hyperedges.partitioner.get) 71 | 72 | val newHyperedges: HyperedgeRDD[ED, VD] = 73 | hyperedges.withPartitionsRDD( 74 | hyperedges.partitionsRDD.zipPartitions(shippedVerts)({ 75 | (hPartIter, shippedVertsIter) => 76 | val ret = hPartIter.map { 77 | case (pid, hyperedgePartition) => { 78 | val newPartition = hyperedgePartition 79 | .updateVertices(shippedVertsIter.filter(_._1 == pid) 80 | .flatMap[(VertexId, VD)]( 81 | _._2.iterator)) 82 | (pid, newPartition) 83 | } 84 | } 85 | ret 86 | }) 87 | ) 88 | 89 | hyperedges = newHyperedges 90 | hasSrcIds = includeSrc 91 | hasDstIds = includeDst 92 | } 93 | } 94 | 95 | def upgradeP(vertices: VertexRDD[VD], includeSrc: Boolean, 96 | includeDst: Boolean, sT: Array[Accumulator[Int]], 97 | zT: Array[Accumulator[Int]], sStart: Array[Accumulator[Long]], 98 | zStart: Array[Accumulator[Long]], sCpl: Array[Accumulator[Long]], 99 | zCpl: Array[Accumulator[Long]], 100 | mrStart: Long) 101 | : Unit = { 102 | val shipSrc = includeSrc && !hasSrcIds 103 | val shipDst = includeDst && !hasDstIds 104 | if (shipSrc || shipDst) { 105 | 106 | val shippedVerts: RDD[(Int, VertexAttributeBlock[VD])] = 107 | vertices.shipVertexAttributesP(shipSrc, shipDst, sT, sStart, sCpl).setName( 108 | ("ReplicatedVertexView.upgrade(%s, %s) - " + 109 | "shippedVerts %s %s (broadcast)") 110 | .format(includeSrc, includeDst, shipSrc, shipDst)) 111 | .partitionBy(hyperedges.partitioner.get) 112 | 113 | val newHyperedges: HyperedgeRDD[ED, VD] = 114 | hyperedges.withPartitionsRDD( 115 | hyperedges.partitionsRDD.zipPartitions(shippedVerts, preservesPartitioning = true) { 116 | (hPartIter, shippedVertsIter) => 117 | val ret = hPartIter.map { 118 | case (pid, hyperedgePartition) => { 119 | val start = System.currentTimeMillis() 120 | zStart(pid) += start 121 | val newPartition = hyperedgePartition 122 | .updateVertices(shippedVertsIter 123 | .flatMap[(VertexId, VD)]( 124 | _._2.iterator)) 125 | zT(pid) += (System.currentTimeMillis() - start).toInt 126 | zCpl(pid) += System.currentTimeMillis() 127 | (pid, newPartition) 128 | } 129 | } 130 | ret 131 | } 132 | ) 133 | 134 | hyperedges = newHyperedges 135 | hasSrcIds = includeSrc 136 | hasDstIds = includeDst 137 | } 138 | } 139 | 140 | /** 141 | * Return a new `ReplicatedVertexView` where the `activeSet` in each 142 | * hyperedge partition 143 | * contains only vertex ids present in `active`. This ships a vertex id 144 | * to all hyperedge partitions 145 | * where it is referenced, ignoring the attribute shipping level. 146 | */ 147 | def withActiveSet(actives: VertexRDD[_]): ReplicatedVertexView[VD, ED] = { 148 | val shippedActives = actives.shipVertexIds() 149 | .setName("ReplicatedVertexView.withActiveSet - shippedActives" + 150 | " (broadcast)") 151 | .partitionBy(hyperedges.partitioner.get) 152 | 153 | val newHyperedges = hyperedges.withPartitionsRDD[ED, 154 | VD](hyperedges.partitionsRDD.zipPartitions(shippedActives, preservesPartitioning = true) { 155 | (hPartIter, shippedActivesIter) => hPartIter.map { 156 | case (pid, hyperedgePartition) => 157 | (pid, hyperedgePartition.withActiveSet(shippedActivesIter 158 | .flatMap(_._2.iterator))) 159 | } 160 | }) 161 | new ReplicatedVertexView(newHyperedges, hasSrcIds, hasDstIds) 162 | } 163 | 164 | def withActiveSetP(actives: VertexRDD[_], start: Array[Accumulator[Long]], 165 | complete: Array[Accumulator[Long]], sT: Array[Accumulator[Int]], 166 | zT: Array[Accumulator[Int]]): ReplicatedVertexView[VD, ED] = { 167 | val shippedActives = actives.shipVertexIdsP(start, sT) 168 | .setName("ReplicatedVertexView.withActiveSet - shippedActives" + 169 | " (broadcast)") 170 | .partitionBy(hyperedges.partitioner.get) 171 | 172 | val newHyperedges = hyperedges.withPartitionsRDD[ED, 173 | VD](hyperedges.partitionsRDD.zipPartitions(shippedActives, preservesPartitioning = true) { 174 | (hPartIter, shippedActivesIter) => hPartIter.map { 175 | case (pid, hyperedgePartition) => 176 | val start = System.currentTimeMillis() 177 | val ret = (pid, hyperedgePartition.withActiveSet(shippedActivesIter 178 | .flatMap(_._2.iterator))) 179 | complete(pid) += System.currentTimeMillis() 180 | zT(pid) += (System.currentTimeMillis() - start).toInt 181 | ret 182 | } 183 | }) 184 | new ReplicatedVertexView(newHyperedges, hasSrcIds, hasDstIds) 185 | } 186 | 187 | /** 188 | * Return a new `ReplicatedVertexView` where vertex attributes in 189 | * hyperedge partition are 190 | * updated using `updates`. This ships a vertex attribute only to the 191 | * hyperedge partitions where 192 | * it is in the position(s) specified by the attribute shipping level. 193 | */ 194 | def updateVertices(updates: VertexRDD[VD]): ReplicatedVertexView[VD, ED] = { 195 | 196 | // send the updated vertex attribute values to its replica partitions 197 | val shippedVerts = updates.shipVertexAttributes(hasSrcIds, hasDstIds) 198 | .setName(("ReplicatedVertexView.updateVertices - " + 199 | "shippedVerts %s %s (broadcast)").format(hasSrcIds, hasDstIds)) 200 | .partitionBy(hyperedges.partitioner.get) 201 | 202 | val newHyperedges = hyperedges.withPartitionsRDD[ED, VD]( 203 | hyperedges.partitionsRDD.zipPartitions(shippedVerts) { 204 | (hPartIter, shippedVertsIter) => hPartIter.map { 205 | case (pid, hyperedgePartition) => 206 | (pid, hyperedgePartition.updateVertices( 207 | shippedVertsIter.filter(_._1 == pid).flatMap{shipped => 208 | shipped._2.iterator})) 209 | } 210 | }) 211 | new ReplicatedVertexView(newHyperedges, hasSrcIds, hasDstIds) 212 | } 213 | } 214 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/RoutingTablePartition.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.impl 19 | 20 | import org.apache.spark.Partitioner 21 | import org.apache.spark.hyperx.impl.RoutingTablePartition.RoutingTableMessage 22 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 23 | import org.apache.spark.hyperx.{PartitionId, VertexId} 24 | import org.apache.spark.rdd.{RDD, ShuffledRDD} 25 | import org.apache.spark.util.collection.{BitSet, PrimitiveVector} 26 | 27 | /* * Forked from GraphX 2.10, modified by Jin Huang */ 28 | private[hyperx] 29 | class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { 30 | /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with 31 | * the given `partitioner`. */ 32 | def copartitionWithVertices(partitioner: Partitioner): 33 | RDD[RoutingTableMessage] = { 34 | new ShuffledRDD[VertexId, Int, Int]( 35 | self, partitioner).setSerializer(new RoutingTableMessageSerializer) 36 | } 37 | } 38 | 39 | private[hyperx] 40 | object RoutingTableMessageRDDFunctions { 41 | import scala.language.implicitConversions 42 | 43 | implicit def rdd2RoutingTableMessageRDDFunctions(rdd: RDD[RoutingTableMessage]) = { 44 | new RoutingTableMessageRDDFunctions(rdd) 45 | } 46 | } 47 | 48 | private[hyperx] 49 | object RoutingTablePartition { 50 | /** 51 | * A message from an edge partition to a vertex specifying the position 52 | * in which the edge 53 | * partition references the vertex (src, dst, 54 | * or both). The edge partition is encoded in the lower 55 | * 30 bytes of the Int, and the position is encoded in the upper 2 bytes 56 | * of the Int. 57 | */ 58 | type RoutingTableMessage = (VertexId, Int) 59 | val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) 60 | 61 | /** Generate a `RoutingTableMessage` for each vertex referenced in 62 | * `hyperedgePartition`. */ 63 | def hyperedgePartitionToMsgs(pid: PartitionId, 64 | hyperedgePartition: HyperedgePartition[_, _]) 65 | : Iterator[RoutingTableMessage] = { 66 | // Determine which positions each vertex id appears in using a map 67 | // where the low 2 bits represent src and dst 68 | val map = new HyperXOpenHashMap[VertexId, Byte] 69 | hyperedgePartition.srcIds.iterator.foreach { srcSet => 70 | srcSet.iterator.foreach { srcId => 71 | map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) 72 | } 73 | } 74 | hyperedgePartition.dstIds.iterator.foreach { dstSet => 75 | dstSet.iterator.foreach { dstId => 76 | map.changeValue(dstId, 0x2, (b: Byte) => (b | 0x2).toByte) 77 | } 78 | } 79 | map.iterator.map { vidAndPosition => 80 | val vid = vidAndPosition._1 81 | val position = vidAndPosition._2 82 | toMessage(vid, pid, position) 83 | } 84 | } 85 | 86 | def flatHyperedgePartitionToMsgs(pid: PartitionId, 87 | hyperedgePartition: FlatHyperedgePartition[_, _]) 88 | : Iterator[RoutingTableMessage] = { 89 | val map = new HyperXOpenHashMap[VertexId, Byte]() 90 | val size = hyperedgePartition.vertexIds.size 91 | (0 until size).foreach{i => 92 | val vid = hyperedgePartition.vertexIds(i) 93 | val flag = hyperedgePartition.srcFlags(i) 94 | if (flag) { 95 | map.changeValue(vid, 0x1, (b: Byte) => (b | 0x1).toByte) 96 | } 97 | else { 98 | map.changeValue(vid, 0x2, (b: Byte) => (b | 0x2).toByte) 99 | } 100 | } 101 | map.map{v=> 102 | toMessage(v._1, pid, v._2) 103 | }.iterator 104 | } 105 | 106 | private def toMessage(vid: VertexId, pid: PartitionId, 107 | position: Byte): RoutingTableMessage = { 108 | val positionUpper2 = position << 30 109 | val pidLower30 = pid & 0x3FFFFFFF 110 | (vid, positionUpper2 | pidLower30) 111 | } 112 | 113 | /** Build a `RoutingTablePartition` from `RoutingTableMessage`s. */ 114 | def fromMsgs(numEdgePartitions: Int, iter: Iterator[RoutingTableMessage]) 115 | : RoutingTablePartition = { 116 | val pid2vid = Array.fill(numEdgePartitions)(new 117 | PrimitiveVector[VertexId]) 118 | val srcFlags = Array.fill(numEdgePartitions)(new 119 | PrimitiveVector[Boolean]) 120 | val dstFlags = Array.fill(numEdgePartitions)(new 121 | PrimitiveVector[Boolean]) 122 | for (msg <- iter) { 123 | val vid = vidFromMessage(msg) 124 | val pid = pidFromMessage(msg) 125 | val position = positionFromMessage(msg) 126 | pid2vid(pid) += vid 127 | srcFlags(pid) += (position & 0x1) != 0 128 | dstFlags(pid) += (position & 0x2) != 0 129 | } 130 | 131 | new RoutingTablePartition(pid2vid.zipWithIndex.map { 132 | case (vids, pid) => (vids.trim().array, toBitSet(srcFlags(pid)), 133 | toBitSet(dstFlags(pid))) 134 | }) 135 | } 136 | 137 | private def vidFromMessage(msg: RoutingTableMessage): VertexId = msg._1 138 | 139 | private[hyperx] def pidFromMessage(msg: RoutingTableMessage): PartitionId = msg._2 & 0x3FFFFFFF 140 | 141 | private def positionFromMessage(msg: RoutingTableMessage): Byte = (msg._2 >> 30).toByte 142 | 143 | /** Compact the given vector of Booleans into a BitSet. */ 144 | private def toBitSet(flags: PrimitiveVector[Boolean]): BitSet = { 145 | val bitset = new BitSet(flags.size) 146 | var i = 0 147 | while (i < flags.size) { 148 | if (flags(i)) { 149 | bitset.set(i) 150 | } 151 | i += 1 152 | } 153 | bitset 154 | } 155 | } 156 | 157 | /** 158 | * Stores the locations of edge-partition join sites for each vertex 159 | * attribute in a particular 160 | * vertex partition. This provides routing information for shipping vertex 161 | * attributes to edge 162 | * partitions. 163 | */ 164 | private[hyperx] 165 | class RoutingTablePartition( 166 | private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) 167 | extends Serializable { 168 | /** The maximum number of edge partitions this `RoutingTablePartition` is 169 | * built to join with. */ 170 | val numHyperedgePartitions: Int = routingTable.size 171 | 172 | /** Returns the number of vertices that will be sent to the specified 173 | * edge partition. */ 174 | def partitionSize(pid: PartitionId): Int = routingTable(pid)._1.size 175 | 176 | /** Returns an iterator over all vertex ids stored in this 177 | * `RoutingTablePartition`. */ 178 | def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1 179 | .iterator) 180 | 181 | /** Returns a new RoutingTablePartition reflecting a reversal of all edge 182 | * directions. */ 183 | def reverse: RoutingTablePartition = { 184 | new RoutingTablePartition(routingTable.map { 185 | case (vids, srcVids, dstVids) => (vids, dstVids, srcVids) 186 | }) 187 | } 188 | 189 | /** 190 | * Runs `f` on each vertex id to be sent to the specified edge partition. 191 | * Vertex ids can be 192 | * filtered by the position they have in the edge partition. 193 | */ 194 | def foreachWithinHyperedgePartition 195 | (pid: PartitionId, includeSrc: Boolean, includeDst: Boolean) 196 | (f: VertexId => Unit) { 197 | val (vidsCandidate, srcVids, dstVids) = routingTable(pid) 198 | // val size = vidsCandidate.length 199 | if (includeSrc && includeDst) { 200 | // Avoid checks for performance 201 | vidsCandidate.iterator.foreach(f) 202 | } else if (!includeSrc && !includeDst) { 203 | // Do nothing 204 | } else { 205 | val relevantVids = if (includeSrc) srcVids else dstVids 206 | relevantVids.iterator.foreach { i => f(vidsCandidate(i))} 207 | } 208 | } 209 | } 210 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/ShippableVertexPartition.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.impl 19 | 20 | import org.apache.spark.Logging 21 | import org.apache.spark.hyperx.{PartitionId, VertexId} 22 | import org.apache.spark.util.collection.{BitSet, PrimitiveVector} 23 | 24 | import scala.reflect.ClassTag 25 | 26 | /** Stores vertex attributes to ship to an edge partition. 27 | * Forked from GraphX 2.10, modified by Jin Huang 28 | * */ 29 | private[hyperx] 30 | class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], 31 | val attrs: Array[VD]) extends Serializable { 32 | def iterator: Iterator[(VertexId, VD)] = 33 | (0 until vids.size).iterator.map { i => (vids(i), attrs(i))} 34 | } 35 | 36 | private[hyperx] 37 | object ShippableVertexPartition extends Logging{ 38 | /** Construct a `ShippableVertexPartition` from the given vertices 39 | * without any routing table. */ 40 | def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]) 41 | : ShippableVertexPartition[VD] = 42 | apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) 43 | 44 | /** 45 | * Construct a `ShippableVertexPartition` from the given vertices with 46 | * the specified routing table, filling in missing vertices mentioned in the 47 | * routing table using `defaultVal`. 48 | */ 49 | def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)], 50 | routingTable: RoutingTablePartition, defaultVal: VD) 51 | : ShippableVertexPartition[VD] = { 52 | val fullIter = iter ++ routingTable.iterator.map(vid => 53 | (vid, defaultVal)) 54 | val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, 55 | (a: VD, b: VD) => a) 56 | new ShippableVertexPartition(index, values, mask, routingTable) 57 | } 58 | 59 | import scala.language.implicitConversions 60 | 61 | /** 62 | * Implicit conversion to allow invoking ``VertexPartitionBase`` operations 63 | * directly on a `ShippableVertexPartition`. 64 | */ 65 | implicit def shippablePartitionToOps[VD: ClassTag]( 66 | partition: ShippableVertexPartition[VD]) = 67 | new ShippableVertexPartitionOps(partition) 68 | 69 | /** 70 | * Implicit evidence that `ShippableVertexPartition` is a member of the 71 | * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking 72 | * `VertexPartitionBase` operations on a `ShippableVertexPartition` via an 73 | * evidence parameter, as in [[VertexPartitionBaseOps]]. 74 | */ 75 | implicit object ShippableVertexPartitionOpsConstructor extends 76 | VertexPartitionBaseOpsConstructor[ShippableVertexPartition] { 77 | def toOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) 78 | : VertexPartitionBaseOps[VD, ShippableVertexPartition] = 79 | shippablePartitionToOps(partition) 80 | } 81 | 82 | } 83 | 84 | /** 85 | * A map from vertex id to vertex attribute that additionally stores edge 86 | * partition join sites for each vertex attribute, enabling joining with an 87 | * [[org.apache.spark.hyperx.HyperedgeRDD]]. 88 | */ 89 | private[hyperx] 90 | class ShippableVertexPartition[VD: ClassTag]( val index: VertexIdToIndexMap, 91 | val values: Array[VD], val mask: BitSet, 92 | val routingTable: RoutingTablePartition) 93 | extends VertexPartitionBase[VD] { 94 | 95 | /** Return a new ShippableVertexPartition with the specified routing 96 | * table. */ 97 | def withRoutingTable(routingTable_ : RoutingTablePartition) 98 | : ShippableVertexPartition[VD] = { 99 | new ShippableVertexPartition(index, values, mask, routingTable_) 100 | } 101 | 102 | /** 103 | * Generate a `VertexAttributeBlock` for each edge partition keyed on the 104 | * edge partition ID. The `VertexAttributeBlock` contains the vertex 105 | * attributes from the current partition that are referenced in the 106 | * specified positions in the edge partition. 107 | */ 108 | def shipVertexAttributes(shipSrc: Boolean,shipDst: Boolean) 109 | : Iterator[(PartitionId, VertexAttributeBlock[VD])] = { 110 | Iterator.tabulate(routingTable.numHyperedgePartitions) { pid => 111 | val initialSize = if (shipSrc && shipDst) routingTable 112 | .partitionSize(pid) else 64 113 | val vids = new PrimitiveVector[VertexId](initialSize) 114 | val attrs = new PrimitiveVector[VD](initialSize) 115 | routingTable.foreachWithinHyperedgePartition(pid, shipSrc, 116 | shipDst) { vid => 117 | if (isDefined(vid)) { 118 | vids += vid 119 | attrs += this(vid) 120 | } 121 | } 122 | (pid, new VertexAttributeBlock(vids.trim().array, 123 | attrs.trim().array)) 124 | } 125 | } 126 | 127 | /** 128 | * Generate a `VertexId` array for each edge partition keyed on the edge 129 | * partition ID. The array contains the visible vertex ids from the current 130 | * partition that are referenced in the edge partition. 131 | */ 132 | def shipVertexIds(): Iterator[(PartitionId, Array[VertexId])] = 133 | Iterator.tabulate(routingTable.numHyperedgePartitions) { pid => 134 | val vids = new PrimitiveVector[VertexId](routingTable.partitionSize 135 | (pid)) 136 | var i = 0 137 | routingTable.foreachWithinHyperedgePartition(pid, includeSrc = true, 138 | includeDst = true) { vid => 139 | if (isDefined(vid)) { 140 | vids += vid 141 | } 142 | i += 1 143 | } 144 | (pid, vids.trim().array) 145 | } 146 | } 147 | 148 | private[hyperx] class ShippableVertexPartitionOps[VD: ClassTag](self: 149 | ShippableVertexPartition[VD]) extends 150 | VertexPartitionBaseOps[VD, ShippableVertexPartition](self) { 151 | 152 | def withIndex(index: VertexIdToIndexMap): ShippableVertexPartition[VD] = { 153 | new ShippableVertexPartition(index, self.values, self.mask, 154 | self.routingTable) 155 | } 156 | 157 | def withValues[VD2: ClassTag](values: Array[VD2]) 158 | : ShippableVertexPartition[VD2] = { 159 | new ShippableVertexPartition( 160 | self.index, values, self.mask, self.routingTable) 161 | } 162 | 163 | def withMask(mask: BitSet): ShippableVertexPartition[VD] = { 164 | new ShippableVertexPartition( 165 | self.index, self.values, mask, self.routingTable) 166 | } 167 | } 168 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/VertexPartition.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.impl 19 | 20 | import org.apache.spark.Logging 21 | import org.apache.spark.hyperx.VertexId 22 | import org.apache.spark.util.collection.BitSet 23 | 24 | import scala.reflect.ClassTag 25 | 26 | /** Forked from GraphX 2.10, modified by Jin Huang */ 27 | private[hyperx] object VertexPartition extends Logging { 28 | /** Construct a `VertexPartition` from the given vertices. */ 29 | def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]) 30 | : VertexPartition[VD] = { 31 | val (index, values, mask) = VertexPartitionBase.initFrom(iter) 32 | new VertexPartition(index, values, mask) 33 | } 34 | 35 | import scala.language.implicitConversions 36 | 37 | /** 38 | * Implicit conversion to allow invoking `VertexPartitionBase` operations 39 | * directly on a `VertexPartition`. 40 | */ 41 | implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) = 42 | new VertexPartitionOps(partition) 43 | 44 | /** 45 | * Implicit evidence that `VertexPartition` is a member of the 46 | * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking 47 | * `VertexPartitionBase` operations on a `VertexPartition` via an 48 | * evidence parameter, as in [[VertexPartitionBaseOps]]. 49 | */ 50 | implicit object VertexPartitionOpsConstructor 51 | extends VertexPartitionBaseOpsConstructor[VertexPartition] { 52 | def toOps[VD: ClassTag](partition: VertexPartition[VD]) 53 | : VertexPartitionBaseOps[VD, VertexPartition] = 54 | partitionToOps(partition) 55 | } 56 | 57 | 58 | } 59 | 60 | /** A map from vertex id to vertex attribute. */ 61 | private[hyperx] class VertexPartition[VD: ClassTag]( 62 | val index: VertexIdToIndexMap, val values: Array[VD], val mask: BitSet) 63 | extends VertexPartitionBase[VD] 64 | 65 | private[hyperx] class VertexPartitionOps[VD: ClassTag]( 66 | self:VertexPartition[VD]) extends 67 | VertexPartitionBaseOps[VD, VertexPartition](self) { 68 | 69 | def withIndex(index: VertexIdToIndexMap): VertexPartition[VD] = { 70 | new VertexPartition(index, self.values, self.mask) 71 | } 72 | 73 | def withValues[VD2: ClassTag](values: Array[VD2]): VertexPartition[VD2] = { 74 | new VertexPartition(self.index, values, self.mask) 75 | } 76 | 77 | def withMask(mask: BitSet): VertexPartition[VD] = { 78 | new VertexPartition(self.index, self.values, mask) 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/VertexPartitionBase.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.impl 19 | 20 | import org.apache.spark.hyperx.VertexId 21 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 22 | import org.apache.spark.util.collection.BitSet 23 | 24 | import scala.language.higherKinds 25 | import scala.reflect.ClassTag 26 | 27 | /** Forked from GraphX 2.10, modified by Jin Huang */ 28 | private[hyperx] object VertexPartitionBase { 29 | /** 30 | * Construct the constituents of a VertexPartitionBase from the given 31 | * vertices, merging duplicate entries arbitrarily. 32 | */ 33 | def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) 34 | : (VertexIdToIndexMap, Array[VD], BitSet) = { 35 | val map = new HyperXOpenHashMap[VertexId, VD] 36 | iter.foreach { pair => 37 | map(pair._1) = pair._2 38 | } 39 | (map.keySet, map._values, map.keySet.getBitSet) 40 | } 41 | 42 | /** 43 | * Construct the constituents of a VertexPartitionBase from the given 44 | * vertices, merging duplicate entries using `mergeFunc`. 45 | */ 46 | def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], 47 | mergeFunc: (VD, VD) => VD) 48 | : (VertexIdToIndexMap, Array[VD], BitSet) = { 49 | val map = new HyperXOpenHashMap[VertexId, VD] 50 | iter.foreach { pair => 51 | map.setMerge(pair._1, pair._2, mergeFunc) 52 | } 53 | (map.keySet, map._values, map.keySet.getBitSet) 54 | } 55 | 56 | } 57 | 58 | /** 59 | * An abstract map from vertex id to vertex attribute. [[VertexPartition]] is 60 | * the corresponding concrete implementation. [[VertexPartitionBaseOps]] 61 | * provides a variety of operations for VertexPartitionBase and subclasses that 62 | * provide implicit evidence of membership in the 63 | * `VertexPartitionBaseOpsConstructor` typeclass (for example 64 | * [[VertexPartition.VertexPartitionOpsConstructor]]). 65 | */ 66 | private[hyperx] abstract class VertexPartitionBase[@specialized(Long, Int, 67 | Double) VD: ClassTag] extends Serializable { 68 | 69 | def index: VertexIdToIndexMap 70 | 71 | def values: Array[VD] 72 | 73 | def mask: BitSet 74 | 75 | // val capacity: Int = index.capacity 76 | 77 | val capacity: Int = mask.capacity 78 | 79 | def size: Int = mask.cardinality() 80 | 81 | /** Return the vertex attribute for the given vertex ID. */ 82 | def apply(vid: VertexId): VD = { 83 | if (index.getPos(vid) < 0) { 84 | } 85 | values(index.getPos(vid)) 86 | } 87 | 88 | def isDefined(vid: VertexId): Boolean = { 89 | val pos = index.getPos(vid) 90 | pos >= 0 && mask.get(pos) 91 | } 92 | 93 | def iterator: Iterator[(VertexId, VD)] = 94 | mask.iterator.map(ind => (index.getValue(ind), values(ind))) 95 | } 96 | 97 | /** 98 | * A typeclass for subclasses of `VertexPartitionBase` representing the 99 | * ability to wrap them in a `VertexPartitionBaseOps`. 100 | */ 101 | private[hyperx] trait VertexPartitionBaseOpsConstructor[T[X] <: 102 | VertexPartitionBase[X]] { 103 | def toOps[VD: ClassTag](partition: T[VD]): VertexPartitionBaseOps[VD, T] 104 | } 105 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/impl/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashSet 4 | 5 | /** 6 | * Collection of implementation classes of HyperX 7 | * 8 | * Forked from GraphX 2.10, modified by Jin Huang 9 | */ 10 | package object impl { 11 | private[hyperx] type VertexIdToIndexMap = HyperXOpenHashSet[VertexId] 12 | // private[hyperx] type VertexIdToIndexMap = HyperXLongBitSet 13 | } 14 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/BetweennessCentrality.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import org.apache.spark.Logging 4 | import org.apache.spark.hyperx._ 5 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 6 | import org.apache.spark.rdd.RDD 7 | 8 | import scala.collection.mutable 9 | import scala.reflect.ClassTag 10 | 11 | /** 12 | * Compute the betweenness centrality for every vertex in the hypergraph 13 | * 14 | * The betweenness centrality is defined on a vertex as the fraction of shortest 15 | * paths between specified two vertices in the hypergraph that passes this 16 | * vertex 17 | * 18 | * The implementation employs a breadth first search first to get the shortest 19 | * paths for every vertex, and then conducts a back propagation along the 20 | * shortest paths to accumulate the centrality incrementally 21 | */ 22 | object BetweennessCentrality extends Logging { 23 | 24 | // sourceId -> (dst, value, precedence[count]) 25 | type BCMap = Map[VertexId, (Int, Int, HyperXOpenHashMap[VertexId, Int])] 26 | 27 | def run[VD: ClassTag, ED: ClassTag] (hypergraph: Hypergraph[VD, ED]) 28 | : RDD[(VertexId, Double)] = { 29 | 30 | // val num = 56 31 | val num = 28 32 | run(hypergraph, hypergraph.pickRandomVertices(num.toInt).toSeq) 33 | } 34 | 35 | 36 | def run[VD: ClassTag, ED: ClassTag] (hypergraph: Hypergraph[VD, ED], 37 | landMarks: Seq[VertexId]): RDD[(VertexId, Double)] = { 38 | 39 | val bcHypergraph = hypergraph.mapVertices((vid, attr) => 40 | if (landMarks.contains(vid)) { 41 | makeBCMap(vid -> (0, 0, new HyperXOpenHashMap[VertexId, Int]())) 42 | } 43 | else { 44 | makeBCMap() 45 | } 46 | ) 47 | 48 | val initialMsg = makeBCMap() 49 | 50 | def vertexProgram(id: VertexId, attr: BCMap, msg: BCMap): BCMap = 51 | mergeMap(attr, msg) 52 | 53 | def hyperedgeProgram(tuple: HyperedgeTuple[BCMap, ED]) 54 | : Iterator[(VertexId, BCMap)] = { 55 | val newAttr = mergeMap( 56 | tuple.srcAttr.map(attr => increase(attr._2, attr._1)).iterator) 57 | 58 | tuple.dstAttr.filter(attr => !is(attr._2, mergeMap(attr._2, newAttr))) 59 | .flatMap(attr => Iterator((attr._1, newAttr))).iterator 60 | // .reduce[Iterator[(VertexId, BCMap)]](_ ++ _) 61 | } 62 | 63 | // breadth first search 64 | val bfsHypergraph = HyperPregel(bcHypergraph, initialMsg)( 65 | vertexProgram, hyperedgeProgram, mergeMap) 66 | 67 | // back propagation from the farthest vertices 68 | val sc = hypergraph.vertices.context 69 | val vertices = bfsHypergraph.vertices.collect() 70 | val vertexBC = sc.accumulableCollection( 71 | mutable.HashMap[VertexId, Double]()) 72 | vertices.foreach{v => vertexBC.value.update(v._1, 0)} 73 | val broadcastVertices = sc.broadcast(vertices) 74 | sc.parallelize(landMarks).foreach{source => 75 | val vertexInfluence = new mutable.HashMap[VertexId, Double]() 76 | val sortedVertices = broadcastVertices.value.filter(v => 77 | v._2.contains(source)).sortBy(v => -v._2(source)._1) 78 | sortedVertices.foreach(v => vertexInfluence.update(v._1, 0)) 79 | sortedVertices.foreach{v => 80 | vertexInfluence(v._1) += 1 81 | v._2(source)._3.foreach{precedent => 82 | val preId = precedent._1 83 | // this could be inefficient due to the linear scan filtering 84 | val tracking = broadcastVertices.value.filter(v => 85 | v._1 == preId)(0)._2(source)._2 86 | vertexInfluence(preId) += 87 | precedent._2 * 1.0 / v._2(source)._2 * 88 | vertexInfluence(v._1) * tracking 89 | } 90 | vertexBC += v._1 -> vertexInfluence(v._1) 91 | } 92 | } 93 | sc.parallelize(vertexBC.value.map(v => (v._1, v._2)).toSeq) 94 | } 95 | 96 | private[hyperx] def makeBCMap(x: (VertexId, (Int, Int, HyperAttr[Int]))*) = 97 | Map(x: _*) 98 | 99 | // private def makeHyperedgeAttr(x: (VertexId, Int)*) = 100 | // Map(x: _*) 101 | 102 | private def increase(map: BCMap, id: VertexId) 103 | : BCMap = { 104 | map.map{case (vid, (dist, value, attr)) => 105 | vid -> (dist + 1, value, if (attr == null && attr.isEmpty) 106 | makeSelfAttr(id) else attr)} 107 | } 108 | 109 | private def mergeMap(maps: Iterator[BCMap]): BCMap = { 110 | maps.reduce(mergeMap) 111 | } 112 | 113 | private def mergeMap(bcMapA: BCMap, bcMapB: BCMap): BCMap = { 114 | val mergedMap = (bcMapA.keySet ++ bcMapB.keySet).map { 115 | k => k ->(math.min(bcMapDist(bcMapA, k), bcMapDist(bcMapB, k)), 0, 116 | new HyperXOpenHashMap[VertexId, Int]()) 117 | }.toMap 118 | updateMap(updateMap(mergedMap, bcMapA), bcMapB) 119 | } 120 | 121 | private def updateMap(bcMapA: BCMap, bcMapB: BCMap): BCMap = { 122 | if (bcMapB.isEmpty) { 123 | bcMapA 124 | } 125 | else { 126 | bcMapA.keySet.map { k => k ->(bcMapDist(bcMapA, k), 127 | if (bcMapDist(bcMapA, k) == bcMapDist(bcMapB, k)) 128 | bcMapVal(bcMapA, k) + bcMapVal(bcMapB, k) 129 | else bcMapVal(bcMapA, k), 130 | if (bcMapDist(bcMapA, k) == bcMapDist(bcMapB, k)) 131 | updateMapAttr(bcMapA, bcMapB, k) 132 | else bcMapAttr(bcMapA, k)) 133 | }.toMap 134 | } 135 | } 136 | 137 | private def updateMapAttr (mapA: BCMap, mapB: BCMap, k: VertexId) 138 | : HyperAttr[Int] = { 139 | if (mapB.nonEmpty && mapB.contains(k)) { 140 | bcMapAttr(mapB, k).foreach(attr => 141 | bcMapAttr(mapA, k).update(attr._1, attr._2 + 142 | bcMapAttr(mapA, k).getOrElse(attr._1, 0))) 143 | } 144 | bcMapAttr(mapA, k) 145 | } 146 | 147 | private def bcMapDist(map: BCMap, key: VertexId): Int = { 148 | map.getOrElse(key, 149 | (Int.MaxValue, 0, null.asInstanceOf[HyperAttr[Int]]))._1 150 | } 151 | 152 | private def bcMapVal(map: BCMap, key: VertexId): Int = { 153 | map.getOrElse(key, 154 | (Int.MaxValue, 0, null.asInstanceOf[HyperAttr[Int]]))._2 155 | } 156 | 157 | private def bcMapAttr(map: BCMap, key: VertexId): HyperAttr[Int]= { 158 | map.getOrElse(key, 159 | (Int.MaxValue, 0, null.asInstanceOf[HyperAttr[Int]]))._3 160 | } 161 | 162 | private def makeSelfAttr(id: VertexId): HyperAttr[Int] = { 163 | val attr = new HyperXOpenHashMap[VertexId, Int]() 164 | attr.update(id, 1) 165 | attr 166 | } 167 | 168 | private[hyperx] def is(a: BCMap, b: BCMap): Boolean = { 169 | if (a.size != b.size) false 170 | else if (a.isEmpty && b.nonEmpty || a.nonEmpty && b.isEmpty) false 171 | else if (a.isEmpty && b.isEmpty) true 172 | else { 173 | val keys = a.map(v => b.contains(v._1)).reduce(_ && _) 174 | if (keys) { 175 | val values = a.map(v => b(v._1)._1 == v._2._1 && 176 | b(v._1)._2 == v._2._2).reduce(_ && _) 177 | if (values) { 178 | val innerSet = a.map(v => 179 | v._2._3.size == b(v._1)._3.size && ((v._2._3.isEmpty && 180 | b(v._1)._3.isEmpty) || (v._2._3.nonEmpty && 181 | b(v._1)._3.nonEmpty && v._2._3.map(i => 182 | b(v._1)._3.hasKey(i._1) && 183 | b(v._1)._3(i._1) == i._2) 184 | .reduce(_ && _))) 185 | ).reduce(_ && _) 186 | innerSet 187 | } 188 | else false 189 | } 190 | else false 191 | } 192 | } 193 | } 194 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/LabelPropagation.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import org.apache.spark.{Accumulator, Logging} 4 | import org.apache.spark.hyperx.{HyperPregel, HyperedgeTuple, Hypergraph, VertexId} 5 | 6 | import scala.reflect.ClassTag 7 | 8 | /** 9 | * Compute the static label propagation on the hypergraph. 10 | * 11 | * The procedure is similar to the label propagation on a graph, 12 | * the difference is as follows: 13 | * 1. The messages are sent and received between every pair of vertices in a 14 | * hyperedge 15 | * 2. The label weight is a fraction depending on the destination vertex set 16 | * size, not 1 17 | * 18 | * Similar to the graph implementation, the procedure may not converge and 19 | * may terminates in 20 | * trivial states, e.g., every vertex is a community 21 | */ 22 | object LabelPropagation extends Logging { 23 | def run[VD: ClassTag, ED: ClassTag] 24 | (hypergraph: Hypergraph[VD, ED], numIter: Int) 25 | : Hypergraph[VertexId, ED] = { 26 | 27 | val lpHypergraph = hypergraph.mapVertices((id, _) => id) 28 | 29 | val sc = lpHypergraph.vertices.context 30 | 31 | def hyperedgeProgram(h: HyperedgeTuple[VertexId, ED], 32 | srcAcc: Accumulator[Int], dstAcc: Accumulator[Int], 33 | srcDAcc: Accumulator[Int], dstDAcc: Accumulator[Int] 34 | ) = { 35 | // val msg = (h.srcAttr.map(_._2) ++ h.dstAttr.map(_._2)).map(v => (v, 1L)).toMap 36 | // (h.srcAttr.keySet.iterator ++ h.dstAttr.keySet.iterator).map(v => (v, msg)) 37 | srcDAcc += h.srcAttr.size 38 | dstDAcc += h.dstAttr.size 39 | var start = System.currentTimeMillis() 40 | val srcMsg = h.srcAttr.map(_._2).groupBy(v => v).mapValues(iter => iter.size).maxBy(_._2)._1 41 | srcAcc += (System.currentTimeMillis() - start).toInt 42 | start = System.currentTimeMillis() 43 | val dstMsg = h.dstAttr.map(_._2).groupBy(v => v).mapValues(iter => iter.size).maxBy(_._2)._1 44 | dstAcc += (System.currentTimeMillis() - start).toInt 45 | h.srcAttr.map(v => (v._1, Map(dstMsg -> 1L))).iterator ++ h.dstAttr.map(v => (v._1, Map(srcMsg -> 1L))).iterator 46 | } 47 | 48 | def mergeMessage(count1: Map[VertexId, Long], 49 | count2: Map[VertexId, Long]) 50 | : Map[VertexId, Long] = { 51 | (count1.keySet ++ count2.keySet).map { i => 52 | val count1Val = count1.getOrElse(i, 0L) 53 | val count2Val = count2.getOrElse(i, 0L) 54 | i -> (count1Val + count2Val) 55 | }.toMap 56 | } 57 | 58 | def vertexProgram(vid: VertexId, attr: VertexId, 59 | message: Map[VertexId, Long]) = { 60 | if (message == null || message.isEmpty) attr else message.maxBy(_._2)._1 61 | } 62 | 63 | val initialMessage: Map[VertexId, Long] = null.asInstanceOf[Map[VertexId, Long]] 64 | 65 | HyperPregel.run[VertexId, ED, Map[VertexId, Long]](sc, lpHypergraph, 66 | initialMessage, maxIterations = numIter)(vprog = vertexProgram, 67 | hprog = hyperedgeProgram, mergeMsg = mergeMessage) 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/LabelPropagationPartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import org.apache.spark.hyperx._ 4 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 5 | import org.apache.spark.rdd.RDD 6 | 7 | import scala.util.Random 8 | 9 | object LabelPropagationPartition { 10 | def run(hypergraph: Hypergraph[_, Int], numIter: Int, numPart: PartitionId) : RDD[String] = { 11 | 12 | def hProg(tuple: HyperedgeTuple[Int, Int]): Iterator[(VertexId, Map[Int, Int])] = { 13 | val pid = tuple.attr 14 | (tuple.srcAttr.keySet.iterator ++ tuple.dstAttr.keySet.iterator).map(v => (v, Map(pid -> 1))) 15 | } 16 | 17 | def combine(a: Map[Int, Int], b: Map[Int, Int]): Map[Int, Int] = { 18 | (a.keySet ++ b.keySet).map { i => 19 | val count1Val = a.getOrElse(i, 0) 20 | val count2Val = b.getOrElse(i, 0) 21 | i -> (count1Val + count2Val) 22 | }.toMap 23 | } 24 | 25 | // def vProg(vid: VertexId, attr: Int, message: Map[Int, Int]) = { 26 | // if (message == null || message.size == 0) attr else choosePid(message) 27 | // } 28 | 29 | var h = hypergraph.mapVertices((id, _) => Random.nextInt(numPart)).cache() 30 | val numH = h.hyperedges.map(each => each.srcIds.size + each.dstIds.size).reduce(_ + _) 31 | val avgH = numH / numPart 32 | h = h.mapTuples{tuple => 33 | val candidates = calculateCandidate(tuple) 34 | val maxId = choosePid(candidates) 35 | (tuple.id, maxId) 36 | // val candidates = (tuple.srcAttr.iterator ++ tuple.dstAttr.iterator).toIterable.groupBy(_._2).map(p => (p._1, p._2.size)) 37 | // val maxVal = candidates.maxBy(_._2)._2 38 | // val maxId = candidates.filter(_._2 == maxVal).map(i => (i, Random.nextInt())).maxBy(_._2)._1._1 39 | // (tuple.id, maxId) 40 | } 41 | 42 | val partitions = h.hyperedges.partitionsRDD.flatMap(part => part._2.iterator.map(h => (h.attr, h.srcIds.size + h.dstIds.size))).collect() 43 | val map = partitions.groupBy(i => i._1).mapValues(_.map(_._2).sum).toMap 44 | println("HYPERX DEBUGGING: map " + map.map(each => each._1 + " : " + each._2).reduce(_ + " ; " + _)) 45 | val preference = new HyperXOpenHashMap[PartitionId, Double]() 46 | (0 until numPart).foreach(i => preference.update(i, calculatePref(avgH, map.getOrElse(i, 0)))) 47 | 48 | var msg = h.mapReduceTuples(hProg, combine) 49 | var activeMsg = msg.count() 50 | 51 | var i = 0 52 | while (activeMsg > 0 && i < numIter) { 53 | 54 | val newVerts = h.vertices.innerJoin(msg){(vid, attr, message) => 55 | if (message == null || message.size == 0) attr else choosePid(message, preference) 56 | // if (message == null || message.isEmpty) attr else message.maxBy(each => each._2 * preference(each._1))._1 57 | 58 | }.cache() 59 | val prevH = h 60 | 61 | h = h.outerJoinVertices(newVerts) { 62 | (vid, old,newOpt) => newOpt.getOrElse(old) 63 | } 64 | 65 | val oldMsg = msg 66 | 67 | h = h.mapTuples{tuple => 68 | val candidates = calculateCandidate(tuple) 69 | val maxId = choosePid(candidates) 70 | (tuple.id, maxId) 71 | }.cache() 72 | // h = h.mapTuples(tuple => (tuple.id, (tuple.srcAttr.iterator ++ tuple.dstAttr.iterator).toIterable.groupBy(_._2).map(p => (p._1, p._2.size)).maxBy(_._2)._1)).cache() 73 | 74 | val partitions = h.hyperedges.partitionsRDD.flatMap(part => part._2.iterator.map(h => (h.attr, h.srcIds.size + h.dstIds.size))).collect() 75 | val map = partitions.groupBy(i => i._1).mapValues(_.map(_._2).sum).toMap 76 | println("HYPERX DEBUGGING: map " + map.map(each => each._1 + " : " + each._2).reduce(_ + " ; " + _)) 77 | (0 until numPart).foreach(i => preference.update(i, calculatePref(avgH, map.getOrElse(i, 0)))) 78 | 79 | msg = h.mapReduceTuples(hProg, combine, Some((newVerts, 80 | HyperedgeDirection.Both))).cache() 81 | activeMsg = msg.count() 82 | 83 | oldMsg.unpersist(blocking = false) 84 | newVerts.unpersist(blocking = false) 85 | prevH.unpersistVertices(blocking = true) 86 | prevH.hyperedges.unpersist(blocking = true) 87 | println("HYPERX DEBUGGING: preference in " + i + " " + preference.map(each => each._1 + " " + each._2).reduce(_ + " ; " + _)) 88 | i += 1 89 | } 90 | 91 | println("HYPERX DEBUGGING: preference " + preference.map(each => each._1 + " " + each._2).reduce(_ + " ; " + _)) 92 | 93 | h.hyperedges.partitionsRDD.flatMap[String]{part => 94 | part._2.tupleIterator(true, true).map{tuple => 95 | tuple.attr + " : " + tuple.srcAttr.map(_._1.toString()).reduce(_ + " " + _) + " : " + tuple.dstAttr.map(_._1.toString()).reduce(_+ " " + _) 96 | } 97 | } 98 | } 99 | 100 | private def calculateCandidate(tuple: HyperedgeTuple[Int, Int]): Map[Int, Int] = { 101 | (tuple.srcAttr.iterator ++ tuple.dstAttr.iterator).toIterable.groupBy(_._2).map(p => (p._1, p._2.size)) 102 | } 103 | 104 | // private def calculateCandidate(tuple: HyperedgeTuple[Int, Int], preference: HyperXOpenHashMap[PartitionId, Double]): Map[Int, Int] = { 105 | // (tuple.srcAttr.iterator ++ tuple.dstAttr.iterator).toIterable.groupBy(_._2).map(p => (p._1, (p._2.size * preference(p._1)).toInt)) 106 | // } 107 | 108 | private def choosePid(map: Map[Int, Int]): PartitionId = { 109 | // val sorted = map.toArray.sortBy(_._2) 110 | // val size = sorted.size 111 | // val sum = sorted.map(_._2).sum 112 | // val cumulative = (0 until size).map(i => (sorted(i)._1, (0 to i).map(j => sorted(j)._2).sum.toDouble / sum)).toArray 113 | // val randomDouble = Random.nextDouble() 114 | // var index = 0 115 | // if (size > 1 && cumulative(0)._2 < randomDouble) { 116 | // index = (1 until size).filter(i => cumulative(i)._2 > randomDouble && cumulative(i - 1)._2 <= randomDouble)(0) 117 | // } 118 | // cumulative(index)._1 119 | val maxVal = map.maxBy(_._2)._2 120 | map.filter(_._2 == maxVal).map(i => (i, Random.nextInt())).maxBy(_._2)._1._1 121 | } 122 | 123 | private def choosePid(map: Map[Int, Int], preference: HyperXOpenHashMap[PartitionId, Double]): PartitionId = { 124 | // val sorted = map.map(each => (each._1, (preference(each._1) * each._2).toInt)).toArray.sortBy(_._2) 125 | // val size = sorted.size 126 | // val sum = sorted.map(_._2).sum 127 | // val cumulative = (0 until size).map(i => (sorted(i)._1, (0 to i).map(j => sorted(j)._2).sum.toDouble / sum)).toArray 128 | // val randomDouble = Random.nextDouble() 129 | // var index = 0 130 | // if (size > 1 && cumulative(0)._2 < randomDouble) { 131 | // index = (1 until size).filter(i => cumulative(i)._2 > randomDouble && cumulative(i - 1)._2 <= randomDouble)(0) 132 | // } 133 | // cumulative(index)._1 134 | map.maxBy(each => each._2 * preference(each._1))._1 135 | } 136 | 137 | private def calculatePref(avg: Int, curr: Int) : Double = { 138 | Math.pow(Math.E, (Math.pow(avg, 2) - Math.pow(curr, 2))/ Math.pow(avg, 2)) 139 | } 140 | } 141 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/MllibRDDFunctions.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import scala.language.implicitConversions 4 | import scala.reflect.ClassTag 5 | 6 | import org.apache.spark.annotation.DeveloperApi 7 | import org.apache.spark.HashPartitioner 8 | import org.apache.spark.SparkContext._ 9 | import org.apache.spark.rdd.RDD 10 | import org.apache.spark.util.Utils 11 | 12 | /** 13 | * Machine learning specific RDD functions. 14 | */ 15 | @DeveloperApi 16 | class MllibRDDFunctions[T: ClassTag](self: RDD[T]) extends Serializable { 17 | 18 | /** 19 | * Returns a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding 20 | * window over them. The ordering is first based on the partition index and then the ordering of 21 | * items within each partition. This is similar to sliding in Scala collections, except that it 22 | * becomes an empty RDD if the window size is greater than the total number of items. It needs to 23 | * trigger a Spark job if the parent RDD has more than one partitions and the window size is 24 | * greater than 1. 25 | */ 26 | def sliding(windowSize: Int): RDD[Array[T]] = { 27 | require(windowSize > 0, s"Sliding window size must be positive, but got $windowSize.") 28 | if (windowSize == 1) { 29 | self.map(Array(_)) 30 | } else { 31 | new SlidingRDD[T](self, windowSize) 32 | } 33 | } 34 | 35 | /** 36 | * Reduces the elements of this RDD in a multi-level tree pattern. 37 | * 38 | * @param depth suggested depth of the tree (default: 2) 39 | * @see [[org.apache.spark.rdd.RDD#reduce]] 40 | */ 41 | def treeReduce(f: (T, T) => T, depth: Int = 2): T = { 42 | require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") 43 | val cleanF = self.context.clean(f) 44 | val reducePartition: Iterator[T] => Option[T] = iter => { 45 | if (iter.hasNext) { 46 | Some(iter.reduceLeft(cleanF)) 47 | } else { 48 | None 49 | } 50 | } 51 | val partiallyReduced = self.mapPartitions(it => Iterator(reducePartition(it))) 52 | val op: (Option[T], Option[T]) => Option[T] = (c, x) => { 53 | if (c.isDefined && x.isDefined) { 54 | Some(cleanF(c.get, x.get)) 55 | } else if (c.isDefined) { 56 | c 57 | } else if (x.isDefined) { 58 | x 59 | } else { 60 | None 61 | } 62 | } 63 | MllibRDDFunctions.fromRDD(partiallyReduced).treeAggregate(Option.empty[T])(op, op, depth) 64 | .getOrElse(throw new UnsupportedOperationException("empty collection")) 65 | } 66 | 67 | /** 68 | * Aggregates the elements of this RDD in a multi-level tree pattern. 69 | * 70 | * @param depth suggested depth of the tree (default: 2) 71 | * @see [[org.apache.spark.rdd.RDD#aggregate]] 72 | */ 73 | def treeAggregate[U: ClassTag](zeroValue: U)( 74 | seqOp: (U, T) => U, 75 | combOp: (U, U) => U, 76 | depth: Int = 2): U = { 77 | require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") 78 | if (self.partitions.size == 0) { 79 | return Utils.clone(zeroValue, self.context.env.closureSerializer.newInstance()) 80 | } 81 | val cleanSeqOp = self.context.clean(seqOp) 82 | val cleanCombOp = self.context.clean(combOp) 83 | val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) 84 | var partiallyAggregated = self.mapPartitions(it => Iterator(aggregatePartition(it))) 85 | var numPartitions = partiallyAggregated.partitions.size 86 | val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) 87 | // If creating an extra level doesn't help reduce the wall-clock time, we stop tree aggregation. 88 | while (numPartitions > scale + numPartitions / scale) { 89 | numPartitions /= scale 90 | val curNumPartitions = numPartitions 91 | partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { (i, iter) => 92 | iter.map((i % curNumPartitions, _)) 93 | }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values 94 | } 95 | partiallyAggregated.reduce(cleanCombOp) 96 | } 97 | } 98 | 99 | @DeveloperApi 100 | object MllibRDDFunctions { 101 | 102 | /** Implicit conversion from an RDD to RDDFunctions. */ 103 | implicit def fromRDD[T: ClassTag](rdd: RDD[T]) = new MllibRDDFunctions[T](rdd) 104 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/RandomWalk.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import org.apache.spark.Logging 4 | import org.apache.spark.hyperx._ 5 | 6 | import scala.collection.mutable 7 | import scala.reflect.ClassTag 8 | 9 | /** 10 | * Run random walk on the hypergraph. 11 | * 12 | * The implementation is similar to the page rank implementation for the 13 | * graph counterpart, the difference is as follows: 14 | * 1. The transition probabilities between vertices are computed based on the 15 | * number of out incident hyperedges of the source vertex and the destination 16 | * vertex set size (the size of the hyperedge) 17 | * 2. The hyperedge program will send multiple messages to the destination 18 | * vertices 19 | * 20 | * The default direction is set to [[HyperedgeDirection.Out]] 21 | */ 22 | object RandomWalk extends Logging { 23 | 24 | def run[VD: ClassTag, ED: ClassTag]( hypergraph: Hypergraph[VD, ED]) 25 | :Hypergraph[Double, _] = { 26 | val num = hypergraph.numVertices 27 | run(hypergraph, 10, hypergraph.pickRandomVertices(num.toInt)) 28 | } 29 | 30 | def run[VD: ClassTag, ED: ClassTag]( hypergraph: Hypergraph[VD, ED], 31 | num: Int, maxIter: Int) 32 | :Hypergraph[Double, _] = { 33 | run(hypergraph, maxIter, hypergraph.pickRandomVertices(num)) 34 | } 35 | 36 | def run[VD: ClassTag, ED: ClassTag](hypergraph: Hypergraph[VD, ED], 37 | numIter: Int, startSet: mutable.HashSet[VertexId], resetProb: Double = 0.15) 38 | : Hypergraph[Double, _] = { 39 | 40 | val walkHypergraph: Hypergraph[(Double, Int), Boolean] = hypergraph 41 | .outerJoinVertices(hypergraph.outIncidents){(vid, vdata, deg) => 42 | deg.getOrElse[Int](0)} 43 | .mapTuples(h => (h.id, null.asInstanceOf[Boolean])) 44 | .mapVertices((id, attr) => (if (startSet.contains(id)) 1.0 else 0.0, attr)) 45 | .cache() 46 | 47 | 48 | def vertexProg(id: VertexId, attr: (Double, Int), msgSum: Double): (Double, Int) = { 49 | if (startSet.contains(id)) { 50 | (resetProb * 1.0 + (1 - resetProb) * msgSum, attr._2) 51 | } else { 52 | ((1 - resetProb) * msgSum, attr._2) 53 | } 54 | } 55 | 56 | def hyperedgeProg(hyperedge: HyperedgeTuple[(Double, Int), Boolean]) 57 | // srcAcc: Accumulator[Int], dstAcc: Accumulator[Int], 58 | // srcDAcc: Accumulator[Int], dstDAcc: Accumulator[Int]) 59 | = { 60 | val dstSize = hyperedge.dstAttr.size 61 | val msgVal = hyperedge.srcAttr.zipWithIndex.map{attr => attr._1._2._1 / attr._1._2._2}.sum / dstSize 62 | if (msgVal > 0.0) 63 | hyperedge.dstAttr.map(attr => (attr._1, msgVal)).toIterator 64 | else 65 | Iterator.empty 66 | } 67 | 68 | def messageCombiner(a: Double, b: Double): Double = a + b 69 | 70 | val initialMessage = 0.0 71 | 72 | val ret = HyperPregel(walkHypergraph, initialMessage, numIter, 73 | activeDirection = HyperedgeDirection.Out)( 74 | vertexProg, hyperedgeProg, messageCombiner) 75 | 76 | ret.mapVertices((id, attr) => attr._1) 77 | 78 | } 79 | 80 | } 81 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/SlidingRDD.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import scala.collection.mutable 4 | import scala.reflect.ClassTag 5 | 6 | import org.apache.spark.{TaskContext, Partition} 7 | import org.apache.spark.rdd.RDD 8 | 9 | private[hyperx] 10 | class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T]) 11 | extends Partition with Serializable { 12 | override val index: Int = idx 13 | } 14 | 15 | /** 16 | * Represents a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding 17 | * window over them. The ordering is first based on the partition index and then the ordering of 18 | * items within each partition. This is similar to sliding in Scala collections, except that it 19 | * becomes an empty RDD if the window size is greater than the total number of items. It needs to 20 | * trigger a Spark job if the parent RDD has more than one partitions. To make this operation 21 | * efficient, the number of items per partition should be larger than the window size and the 22 | * window size should be small, e.g., 2. 23 | * 24 | * @param parent the parent RDD 25 | * @param windowSize the window size, must be greater than 1 26 | * 27 | * @see [[org.apache.spark.mllib.rdd.RDDFunctions#sliding]] 28 | */ 29 | private[hyperx] 30 | class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int) 31 | extends RDD[Array[T]](parent) { 32 | 33 | require(windowSize > 1, s"Window size must be greater than 1, but got $windowSize.") 34 | 35 | override def compute(split: Partition, context: TaskContext): Iterator[Array[T]] = { 36 | val part = split.asInstanceOf[SlidingRDDPartition[T]] 37 | (firstParent[T].iterator(part.prev, context) ++ part.tail) 38 | .sliding(windowSize) 39 | .withPartial(false) 40 | .map(_.toArray) 41 | } 42 | 43 | override def getPreferredLocations(split: Partition): Seq[String] = 44 | firstParent[T].preferredLocations(split.asInstanceOf[SlidingRDDPartition[T]].prev) 45 | 46 | override def getPartitions: Array[Partition] = { 47 | val parentPartitions = parent.partitions 48 | val n = parentPartitions.size 49 | if (n == 0) { 50 | Array.empty 51 | } else if (n == 1) { 52 | Array(new SlidingRDDPartition[T](0, parentPartitions(0), Seq.empty)) 53 | } else { 54 | val n1 = n - 1 55 | val w1 = windowSize - 1 56 | // Get the first w1 items of each partition, starting from the second partition. 57 | val nextHeads = 58 | parent.context.runJob(parent, (iter: Iterator[T]) => iter.take(w1).toArray, 1 until n, true) 59 | val partitions = mutable.ArrayBuffer[SlidingRDDPartition[T]]() 60 | var i = 0 61 | var partitionIndex = 0 62 | while (i < n1) { 63 | var j = i 64 | val tail = mutable.ListBuffer[T]() 65 | // Keep appending to the current tail until appended a head of size w1. 66 | while (j < n1 && nextHeads(j).size < w1) { 67 | tail ++= nextHeads(j) 68 | j += 1 69 | } 70 | if (j < n1) { 71 | tail ++= nextHeads(j) 72 | j += 1 73 | } 74 | partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail) 75 | partitionIndex += 1 76 | // Skip appended heads. 77 | i = j 78 | } 79 | // If the head of last partition has size w1, we also need to add this partition. 80 | if (nextHeads.last.size == w1) { 81 | partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(n1), Seq.empty) 82 | } 83 | partitions.toArray 84 | } 85 | } 86 | 87 | // TODO: Override methods such as aggregate, which only requires one Spark job. 88 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/lib/SpectralLearning.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.lib 2 | 3 | import breeze.linalg.svd.SVD 4 | import breeze.linalg.{sum, svd, DenseMatrix => BDM, DenseVector => BDV, Matrix => BM, Vector => BV} 5 | import org.apache.spark.SparkContext._ 6 | import org.apache.spark.hyperx.Hypergraph 7 | import org.apache.spark.hyperx.util.collection.HyperXPrimitiveVector 8 | import org.apache.spark.rdd.RDD 9 | import org.apache.spark.{Logging, SparkContext} 10 | 11 | import scala.reflect.ClassTag 12 | import scala.util.Random 13 | 14 | /** 15 | * Given a hypergraph, first compute its normalized Laplacian, then computes 16 | * the largest k eigen vectors and eigen values 17 | * (via Lanczos-Selective Orthogonalization) for clustering and embedding 18 | * 19 | * Only supports int vertices 20 | */ 21 | object SpectralLearning extends Logging{ 22 | 23 | type VertexSparseMatrix = RDD[(VID, (Array[VID], Array[Double]))] 24 | type VertexDenseMatrix = RDD[(VID, Array[Double])] 25 | type VID = Int 26 | 27 | var sc: SparkContext = _ 28 | 29 | def run[VD: ClassTag, ED: ClassTag](hypergraph: Hypergraph[VD, ED], 30 | eigenK: Int, numIter: Int, tol: Double): (Array[Double], Array[Double]) = { 31 | sc = hypergraph.vertices.context 32 | val laplacian = hypergraph.laplacian.map(each => (each._1.toInt, (each._2._1.map(_.toInt), each._2._2))).cache() 33 | lanczos(laplacian, eigenK, numIter, tol) 34 | } 35 | 36 | def runMRT[VD: ClassTag, ED: ClassTag](hypergraph: Hypergraph[VD, ED], 37 | eigenK: Int, numIter: Int, tol: Double): (Array[Double], Array[Double]) = { 38 | mrtLanczos(hypergraph, eigenK, numIter, tol) 39 | } 40 | 41 | def runTest[VD: ClassTag, ED: ClassTag](hypergraph: Hypergraph[VD, ED]): Unit = { 42 | sc = hypergraph.vertices.context 43 | val laplacian = hypergraph.laplacian.map(each => (each._1.toInt, (each._2._1.map(_.toInt), each._2._2))).cache() 44 | val result = matVecMult(laplacian, new BDV[Double](Array(1.0, 2.0, 3.0, 4.0))) 45 | 46 | logInfo("HYPERX DEBUGGING: result " + result.toArray.map(_.toString).reduce(_ + " " + _)) 47 | 48 | val spectralH = hypergraph.outerJoinVertices(hypergraph.incidents){(vid, vdata, deg) => 49 | deg match { 50 | case someDeg: Some[Int] => 51 | 1.0 / Math.sqrt(someDeg.get) 52 | case None => 53 | 0.0 54 | }}.cache() 55 | val mrtResult = matVecMult(spectralH, new BDV[Double](Array(1.0, 2.0, 3.0, 4.0))) 56 | logInfo("HYPERX DEBUGGING: mrt result " + mrtResult.toArray.map(_.toString).reduce(_ + " " + _)) 57 | } 58 | 59 | private def mrtLanczos[VD: ClassTag, ED: ClassTag](h: Hypergraph[VD, ED], eigenK: Int, numIter: Int, tol: Double): (Array[Double], Array[Double]) = { 60 | val spectralH = h.outerJoinVertices(h.incidents){(vid, vdata, deg) => 61 | deg match { 62 | case someDeg: Some[Int] => 63 | 1.0 / Math.sqrt(someDeg.get) 64 | case None => 65 | 0.0 66 | }}.cache() 67 | 68 | val n = spectralH.vertices.count().toInt 69 | 70 | val alpha, beta = new HyperXPrimitiveVector[Double]() 71 | var alphaLast, alphaCur, betaLast, betaCur = 0.0 72 | var vLast, vCur = BDV.zeros[Double](n) 73 | val b = new BDV[Double]((0 until n).map(i => Random.nextDouble()).toArray) 74 | // val b = new BDV[Double](Array.fill(n)(2.0)) 75 | vCur = (b / norm(b)).asInstanceOf[BDV[Double]] 76 | var vMatrix = new BDM[Double](n, 1, vCur.toArray) 77 | var i = 0 78 | 79 | while((i == 0 || Math.abs(betaCur - 0.0) > tol) && i < numIter) { 80 | val start = System.currentTimeMillis() 81 | val dimension = i + 1 82 | var v = matVecMult(spectralH, vCur) 83 | alphaCur = sum(vCur :* v) 84 | v = v - vCur * alphaCur 85 | v = v - vLast * betaLast 86 | betaCur = norm(v) 87 | alpha += alphaCur 88 | alphaLast = alphaCur 89 | 90 | val alphaVector = new BDV[Double](alpha.array.slice(0, i + 1)) 91 | val betaVector = new BDV[Double]((beta.array.slice(0, i).iterator ++ Iterator(betaCur)).toArray) 92 | 93 | val t = tridiongonal(alphaVector, betaVector) 94 | val (_, eigQ) = Linalg.localEig(t) 95 | 96 | var reflag = false 97 | (0 to i).foreach{j => 98 | if (Math.abs(betaCur * eigQ.toArray(j * dimension + i)) <= Math.sqrt(tol) * l2Norm(t)) { 99 | reflag = true 100 | val rMatrix = vMatrix * new BDM[Double](dimension, 1, eigQ.toArray.slice(j * 101 | dimension, (j + 1) * dimension)) 102 | val r = new BDV[Double](rMatrix.asInstanceOf[BDM[Double]].toArray) 103 | v = v - r * sum(r :* v) 104 | } 105 | } 106 | 107 | if (reflag) { 108 | betaCur = norm(v) 109 | } 110 | beta += betaCur 111 | betaLast = betaCur 112 | vLast = vCur 113 | vCur = v / betaCur 114 | i += 1 115 | vMatrix = new BDM[Double](n, i + 1, vMatrix.toArray ++ vCur.toArray) 116 | val duration = System.currentTimeMillis() - start 117 | println(s"HYPERX: iterator $i with beta $betaCur in $duration ms") 118 | } 119 | 120 | val alphaVector = new BDV[Double](alpha.array.slice(0, i)) 121 | val betaVector = new BDV[Double](beta.array.slice(0, i)) 122 | val t = tridiongonal(alphaVector, betaVector) 123 | val (eigV, eigQ) = Linalg.localEig(t) 124 | vMatrix = new BDM[Double](n, i, vMatrix.toArray.slice(0, i * n)) 125 | val retVal = eigV.toArray.slice(0, eigenK) 126 | val retVec = (vMatrix * new BDM[Double](i, eigenK, eigQ.toArray.slice(0, eigenK * i))).asInstanceOf[BDM[Double]].toArray 127 | (retVal, retVec) 128 | } 129 | 130 | private def lanczos(matrix: VertexSparseMatrix, eigenK: Int, numIter: Int, tol: Double): (Array[Double], Array[Double]) = { 131 | val n = matrix.count().toInt 132 | 133 | val alpha, beta = new HyperXPrimitiveVector[Double]() 134 | var alphaLast, alphaCur, betaLast, betaCur = 0.0 135 | var vLast, vCur = BDV.zeros[Double](n) 136 | val b = new BDV[Double]((0 until n).map(i => Random.nextDouble()).toArray) 137 | // val b = new BDV[Double](Array.fill(n)(2.0)) 138 | vCur = (b / norm(b)).asInstanceOf[BDV[Double]] 139 | var vMatrix = new BDM[Double](n, 1, vCur.toArray) 140 | var i = 0 141 | 142 | while((i == 0 || Math.abs(betaCur - 0.0) > tol) && i < numIter) { 143 | val start = System.currentTimeMillis() 144 | val dimension = i + 1 145 | var v = matVecMult(matrix, vCur) 146 | alphaCur = sum(vCur :* v) 147 | v = v - vCur * alphaCur 148 | v = v - vLast * betaLast 149 | betaCur = norm(v) 150 | alpha += alphaCur 151 | alphaLast = alphaCur 152 | 153 | val alphaVector = new BDV[Double](alpha.array.slice(0, i + 1)) 154 | val betaVector = new BDV[Double]((beta.array.slice(0, i).iterator ++ Iterator(betaCur)).toArray) 155 | 156 | val t = tridiongonal(alphaVector, betaVector) 157 | val (_, eigQ) = Linalg.localEig(t) 158 | 159 | var reflag = false 160 | (0 to i).foreach{j => 161 | if (Math.abs(betaCur * eigQ.toArray(j * dimension + i)) <= Math.sqrt(tol) * l2Norm(t)) { 162 | reflag = true 163 | val rMatrix = vMatrix * new BDM[Double](dimension, 1, eigQ.toArray.slice(j * 164 | dimension, (j + 1) * dimension)) 165 | val r = new BDV[Double](rMatrix.asInstanceOf[BDM[Double]].toArray) 166 | v = v - r * sum(r :* v) 167 | } 168 | } 169 | 170 | if (reflag) { 171 | betaCur = norm(v) 172 | } 173 | beta += betaCur 174 | betaLast = betaCur 175 | vLast = vCur 176 | vCur = v / betaCur 177 | i += 1 178 | vMatrix = new BDM[Double](n, i + 1, vMatrix.toArray ++ vCur.toArray) 179 | val duration = System.currentTimeMillis() - start 180 | println(s"HYPERX: iterator $i with beta $betaCur in $duration ms") 181 | } 182 | 183 | val alphaVector = new BDV[Double](alpha.array.slice(0, i)) 184 | val betaVector = new BDV[Double](beta.array.slice(0, i)) 185 | val t = tridiongonal(alphaVector, betaVector) 186 | val (eigV, eigQ) = Linalg.localEig(t) 187 | vMatrix = new BDM[Double](n, i, vMatrix.toArray.slice(0, i * n)) 188 | val retVal = eigV.toArray.slice(0, eigenK) 189 | val retVec = (vMatrix * new BDM[Double](i, eigenK, eigQ.toArray.slice(0, eigenK * i))).asInstanceOf[BDM[Double]].toArray 190 | (retVal, retVec) 191 | } 192 | 193 | private def tridiongonal(alpha: BV[Double], beta: BV[Double]): BM[Double] = { 194 | val n = alpha.size 195 | val values = (0 until n).flatMap{col => 196 | (0 until n).map{row => 197 | if (col == row) { 198 | alpha(col) 199 | } else if (col == row - 1) { 200 | beta(col) 201 | } else if (col == row + 1) { 202 | beta(row) 203 | } else 0.0 204 | } 205 | }.toArray 206 | new BDM[Double](n, n, values) 207 | } 208 | 209 | private def norm(vector: BV[Double]): Double = { 210 | Math.sqrt(vector.valuesIterator.map(d => Math.pow(d, 2)).sum) 211 | } 212 | 213 | private def l2Norm(matrix: BM[Double]): Double = { 214 | val SVD(_, s, _) = svd(matrix.asInstanceOf[BDM[Double]]) 215 | s.toArray.max 216 | } 217 | 218 | private def matVecMult(matrix: VertexSparseMatrix, vector: BDV[Double]): BDV[Double] = { 219 | val map = matrix.filter(row => row._2 != null && row._2._1 != null).flatMap{row => 220 | val rowId = row._1 221 | val rowSize = row._2._1.size 222 | val colArray = new HyperXPrimitiveVector[(VID, Double)]() 223 | val rowSum = (0 until rowSize).map{i => 224 | val colId = row._2._1(i) 225 | val oldVal = row._2._2(i) 226 | colArray += (colId, oldVal * vector(rowId)) 227 | oldVal * vector(colId) 228 | }.sum 229 | Iterator((rowId, rowSum)) ++ colArray.trim().array.iterator 230 | }.reduceByKey(_ + _).collectAsMap() 231 | new BDV[Double]((0 until vector.length).map(i => map.getOrElse(i, 0.0)).toArray) 232 | } 233 | 234 | private def matVecMult(h: Hypergraph[Double, _], vector: BDV[Double]): BDV[Double] = { 235 | val start = System.currentTimeMillis() 236 | val ret = h.mapReduceTuples[Double]({tuple => 237 | val weight = tuple.attr match { 238 | case doubleWeight: Double => 239 | doubleWeight 240 | case _ => 241 | 1.0 242 | } 243 | 244 | val data = (tuple.srcAttr.map(v => v._2 * vector(v._1.toInt)).iterator ++ tuple.dstAttr.map(v => v._2 * vector(v._1.toInt))).sum * weight 245 | (tuple.srcAttr.iterator ++ tuple.dstAttr.iterator).map{u => 246 | (u._1, u._2 * data) 247 | } 248 | }, {(a, b) => a + b}).mapValues((vid, vval) => vector(vid.toInt) - 0.5 * vval).collectAsMap() 249 | logInfo("HYPERX DEBUGGING: mrt " + (System.currentTimeMillis() - start)) 250 | new BDV[Double]((0 until vector.length).map(i => ret.getOrElse(i.toLong, 0.0)).toArray) 251 | } 252 | } 253 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark 2 | 3 | import org.apache.spark.hyperx.util.collection.{HyperXOpenHashMap, 4 | HyperXOpenHashSet} 5 | 6 | /** 7 | * ALPHA COMPONENT 8 | * HyperX is a hypergraph processing framework built on top of Spark and 9 | * extending GraphX 10 | * 11 | * Forked from GraphX 2.10, modified by Jin Huang 12 | */ 13 | 14 | package object hyperx { 15 | 16 | /** 17 | * A 64-bit vertex identifier that uniquely identifies a vertex within a 18 | * hypergraph. 19 | * It does not need to follow any ordering or any constraints other than 20 | * uniqueness. 21 | */ 22 | // type VertexId = Long 23 | 24 | type VertexId = Long 25 | 26 | type HyperedgeId = Int 27 | 28 | /** Integer identifier of a hypergraph partition. */ 29 | type PartitionId = Int 30 | 31 | /** A hash set containing a collection of vertex identifiers */ 32 | // private[hyperx] type VertexSet = OpenHashSet[VertexId] 33 | 34 | private[hyperx] type VertexSet = HyperXOpenHashSet[VertexId] 35 | 36 | /** A hash map containing a collection of vertex identifier and 37 | * corresponding attributes */ 38 | private[hyperx] type HyperAttr[VD] = HyperXOpenHashMap[VertexId, VD] 39 | } 40 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/AwetoPartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.hyperx.util.HyperUtils 4 | import org.apache.spark.rdd.RDD 5 | 6 | import scala.util.Random 7 | 8 | class AwetoPartition extends PartitionStrategy{ 9 | override private[partition] def search(input: RDD[String]): Unit = { 10 | // randomly assign vertices 11 | vRDD = input.flatMap(h => HyperUtils.iteratorFromHString(h)) 12 | .distinct(k).map(v => (v, Random.nextInt(k))).cache() 13 | 14 | // greedily assign hyperedges 15 | hRDD = input.mapPartitions{part => 16 | val degrees = Array.fill(k)(0) 17 | part.map{h => 18 | val pid = (0 until k).map(i => 19 | (i, degrees(i))).minBy(_._2)._1 20 | degrees(pid) += HyperUtils.countDegreeFromHString(h) 21 | (h, pid) 22 | }}.cache() 23 | 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/BipartitePartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.hyperx.util.HyperUtils 4 | import org.apache.spark.rdd.RDD 5 | 6 | import scala.util.Random 7 | 8 | class BipartitePartition extends PartitionStrategy { 9 | override private[partition] def search(input: RDD[String]): Unit = { 10 | // arbitrarily assign hyperedges 11 | hRDD = input.map(h => (h, Random.nextInt(k))).cache() 12 | 13 | // randomly assign vertices 14 | vRDD = hRDD.flatMap(h => HyperUtils.iteratorFromHString(h._1)) 15 | .distinct(k).map(v => (v, Random.nextInt(k))).cache() 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/GreedyPartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.SparkContext._ 4 | import org.apache.spark.hyperx.VertexId 5 | import org.apache.spark.hyperx.util.HyperUtils 6 | import org.apache.spark.rdd.RDD 7 | import scala.collection.mutable 8 | 9 | 10 | class GreedyPartition extends PartitionStrategy{ 11 | override private[partition] def search(input: RDD[String]): Unit = { 12 | hRDD = input.coalesce(k, shuffle = true) 13 | .mapPartitionsWithIndex({(i, p) => 14 | val demands = Array.fill(k)(new mutable.HashSet[VertexId]()) 15 | val degrees = Array.fill(k)(0) 16 | val onepass = p.map{h => 17 | val pid = (0 until k).map(i => 18 | (i, demands(i).size * costDemand + 19 | degrees(i) * costDegree)).minBy(_._2)._1 20 | 21 | HyperUtils.iteratorFromHString(h).foreach(demands(pid).add) 22 | degrees(pid) += HyperUtils.countDegreeFromHString(h) 23 | (h, pid) 24 | } 25 | // todo: a loop 26 | onepass.map{h => 27 | val count = HyperUtils.countDegreeFromHString(h._1) 28 | val extraDemand = (0 until k).map(i => 29 | count - HyperUtils.iteratorFromHString(h._1) 30 | .count(demands(i).contains)) 31 | val newPid = ((0 until k).filter(_ != h._2).map(i => 32 | (i, (demands(i).size + extraDemand(i)) * costDemand + 33 | (degrees(i) + count) * costDegree)).toIterator ++ 34 | Iterator((h._2, demands(h._2).size * costDemand + 35 | degrees(h._2) * costDegree))) 36 | .minBy(_._2)._1 37 | if (newPid != h._2) { 38 | degrees(h._2) -= count 39 | HyperUtils.iteratorFromHString(h._1).foreach( 40 | demands(h._2).remove) 41 | 42 | degrees(newPid) += count 43 | HyperUtils.iteratorFromHString(h._1).foreach( 44 | demands(newPid).add) 45 | } 46 | (h._1, newPid) 47 | } 48 | }).cache() 49 | 50 | val demands = hRDD.map(h => 51 | Tuple2(h._2, HyperUtils.iteratorFromHString(h._1).toSet)) 52 | .reduceByKey(_.union(_)).collect() 53 | val broadcastDemands = hRDD.context.broadcast(demands) 54 | 55 | vRDD = hRDD.flatMap(h => HyperUtils.iteratorFromHString(h._1)) 56 | .distinct(k).mapPartitionsWithIndex{(i, p) => 57 | val locals = Array.fill(k)(0) 58 | p.map{v => 59 | val pid = (0 until k).map(i => 60 | (i, (if (broadcastDemands.value(i)._2.contains(v)) 1 61 | else 0) * costReplica - locals(i))).maxBy(_._2)._1 62 | locals(pid) += 1 63 | (v, pid) 64 | }.toIterator 65 | }.cache() 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/PartitionStrategy.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.{HashPartitioner, Logging} 4 | import org.apache.spark.hyperx._ 5 | import org.apache.spark.hyperx.util.HyperUtils 6 | import org.apache.spark.rdd.RDD 7 | import org.apache.spark.SparkContext._ 8 | /** 9 | * Represents the way hyperedges and vertices are assigned to partitions. 10 | * 11 | * Forked from GraphX 2.10, modified by Jin Huang 12 | */ 13 | trait PartitionStrategy extends Serializable with Logging { 14 | 15 | private[partition] def search(input: RDD[String]): Unit 16 | 17 | def partition(numParts: PartitionId, input: RDD[String]) 18 | : (RDD[(PartitionId, VertexId)], RDD[(PartitionId, String)]) = { 19 | val start = System.currentTimeMillis() 20 | k = numParts 21 | search(input) 22 | logInfo("HYPERX PARTITION: partition in %d ms".format( 23 | System.currentTimeMillis() - start)) 24 | printStatistics() 25 | (vRDD.map(v => (v._2, v._1)), hRDD.map(h => (h._2, h._1))) 26 | } 27 | 28 | def getPartitioner: VertexPartitioner = { 29 | VertexPartitioner(k, vRDD.collect().iterator) 30 | } 31 | 32 | private[partition] var vRDD: RDD[(VertexId, PartitionId)] = _ 33 | private[partition] var hRDD: RDD[(String, PartitionId)] = _ 34 | 35 | def setPartitionParams(degreeCost: Double, replicaCost: Double, 36 | demandCost: Double, normSpace: Int, effSrc: Double, effDst: Double) = { 37 | costDegree = degreeCost 38 | costReplica = replicaCost 39 | costDemand = demandCost 40 | norm = normSpace 41 | effectiveSrc = effSrc 42 | effectiveDst = effDst 43 | } 44 | 45 | private def printStatistics(): Unit = { 46 | 47 | val numH = hRDD.count() 48 | val numV = vRDD.count() 49 | logInfo("HYPERX PARTITION: hyperedges %d vertices %d".format(numH, numV)) 50 | 51 | // demands 52 | val demands = hRDD.map(h => 53 | Tuple2(h._2, HyperUtils.iteratorFromHString(h._1).toSet)) 54 | .reduceByKey(_.union(_)).partitionBy(new HashPartitioner(k)).cache() 55 | val locals = vRDD.map(v => Tuple2(v._2, Set(v._1))).reduceByKey(_ ++ _).partitionBy(new HashPartitioner(k)).cache() 56 | 57 | logInfo("HYPERX DEBUGGING: demands " + demands.map(each => each._1 + " : " + each._2.size).reduce(_ + " ; " + _ )) 58 | logInfo("HYPERX DEBUGGING: locals " + locals.map(each => each._1 + " : " + each._2.size).reduce(_ + " ; " + _)) 59 | 60 | val replicas = demands.zipPartitions(locals){(d, l) => 61 | val dSet = d.filter(_._2.size > 0).map(_._2).reduce(_ ++ _) 62 | val lSet = l.filter(_._2.size > 0).map(_._2).reduce(_ ++ _) 63 | Iterator(dSet.size - dSet.intersect(lSet).size) 64 | }.cache() 65 | 66 | logArray("replicas", replicas.collect()) 67 | logArray("demands", demands.map(_._2.size).collect()) 68 | logArray("locals", locals.map(_._2.size).collect()) 69 | logInfo("HYPERX PARTITION: replicaFactor: %f".format(replicas.sum / vRDD.count())) 70 | // degrees 71 | val degrees = hRDD.map{h => 72 | val pair = HyperUtils.countDetailDegreeFromHString(h._1) 73 | Tuple2(h._2, pair)} 74 | .reduceByKey((a, b) => (a._1 + b._1, a._2 + b._2)).map(_._2) 75 | .collect() 76 | val srcDegrees = degrees.map(_._1) 77 | val dstDegrees = degrees.map(_._2) 78 | logArray("source degrees", srcDegrees) 79 | logArray("destination degrees", dstDegrees) 80 | 81 | demands.unpersist(blocking = false) 82 | locals.unpersist(blocking = false) 83 | replicas.unpersist(blocking = false) 84 | 85 | // val localsCount = vRDD.map(v => Tuple2(v._2, 1)) 86 | // .reduceByKey(_ + _).map(_._2).collect() 87 | } 88 | 89 | private def logArray(name: String, ary: Array[Int]): Unit = { 90 | logInfo("HYPERX PARTITION: %s avg %d min %d max %d std %d std percent %f" 91 | .format(name, HyperUtils.avg(ary).toInt, ary.min, ary.max, 92 | HyperUtils.dvt(ary).toInt, HyperUtils.dvt(ary) / HyperUtils.avg(ary) 93 | )) 94 | } 95 | 96 | private[partition] var k: Int = 0 97 | } 98 | 99 | object PartitionStrategy extends Logging { 100 | def printStatistics(hRDD: RDD[(String, PartitionId)], vRDD: RDD[(VertexId, PartitionId)], k: Int): Unit = { 101 | 102 | val numH = hRDD.count() 103 | val numV = vRDD.count() 104 | logInfo("HYPERX PARTITION: hyperedges %d vertices %d".format(numH, numV)) 105 | 106 | // demands 107 | val demands = hRDD.map(h => 108 | Tuple2(h._2, HyperUtils.iteratorFromHString(h._1).toSet)) 109 | .reduceByKey(_.union(_)).partitionBy(new HashPartitioner(k)).cache() 110 | val locals = vRDD.map(v => Tuple2(v._2, Set(v._1))).reduceByKey(_ ++ _).partitionBy(new HashPartitioner(k)).cache() 111 | 112 | val replicas = demands.zipPartitions(locals){(d, l) => 113 | val dSet = d.filter(_._2.size > 0).map(_._2).reduce(_ ++ _) 114 | val lSet = l.filter(_._2.size > 0).map(_._2).reduce(_ ++ _) 115 | Iterator(dSet.size - dSet.intersect(lSet).size) 116 | }.cache() 117 | 118 | logArray("replicas", replicas.collect()) 119 | logArray("demands", demands.map(_._2.size).collect()) 120 | logArray("locals", locals.map(_._2.size).collect()) 121 | logInfo("HYPERX PARTITION: replicaFactor: %f".format(replicas.sum / vRDD.count())) 122 | // degrees 123 | val degrees = hRDD.map{h => 124 | val pair = HyperUtils.countDetailDegreeFromHString(h._1) 125 | Tuple2(h._2, pair)} 126 | .reduceByKey((a, b) => (a._1 + b._1, a._2 + b._2)).map(_._2) 127 | .collect() 128 | val srcDegrees = degrees.map(_._1) 129 | val dstDegrees = degrees.map(_._2) 130 | logArray("source degrees", srcDegrees) 131 | logArray("destination degrees", dstDegrees) 132 | 133 | demands.unpersist(blocking = false) 134 | locals.unpersist(blocking = false) 135 | replicas.unpersist(blocking = false) 136 | 137 | // val localsCount = vRDD.map(v => Tuple2(v._2, 1)) 138 | // .reduceByKey(_ + _).map(_._2).collect() 139 | } 140 | 141 | private def logArray(name: String, ary: Array[Int]): Unit = { 142 | logInfo("HYPERX PARTITION: %s avg %d min %d max %d std %d std percent %f" 143 | .format(name, HyperUtils.avg(ary).toInt, ary.min, ary.max, 144 | HyperUtils.dvt(ary).toInt, HyperUtils.dvt(ary) / HyperUtils.avg(ary) 145 | )) 146 | } 147 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/PlainPartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.hyperx.util.HyperUtils 4 | import org.apache.spark.rdd.RDD 5 | 6 | import scala.util.Random 7 | 8 | class PlainPartition extends PartitionStrategy { 9 | override def search(input: RDD[String]): Unit = { 10 | hRDD = input.coalesce(k, shuffle = true) 11 | .mapPartitionsWithIndex({(i, p) => 12 | p.map(s => (s, i)) 13 | }, preservesPartitioning = true) 14 | vRDD = hRDD.flatMap(h => 15 | HyperUtils.iteratorFromHString(h._1)).distinct(k) 16 | .map(v => (v, Random.nextInt(k))) 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/ReplicaPartition.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.hyperx.VertexId 4 | import org.apache.spark.hyperx.util.HyperUtils 5 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashSet 6 | import org.apache.spark.rdd.RDD 7 | 8 | import scala.util.Random 9 | 10 | /** 11 | * Created by soone on 11/25/14. 12 | */ 13 | class ReplicaPartition extends PartitionStrategy{ 14 | override private[partition] def search(input: RDD[String]): Unit = { 15 | 16 | hRDD = input.coalesce(1).mapPartitions{p => 17 | val sets = Array.fill(k)(new HyperXOpenHashSet[VertexId]()) 18 | val degrees = Array.fill(k)(0) 19 | p.map { h => 20 | val hSet = HyperUtils.setFromHString(h) 21 | val pid = (0 until k).map(i => (i, sets(i).iterator.count(hSet.contains) * costReplica - costDegree * degrees(i))).maxBy(_._2)._1 22 | hSet.iterator.foreach(sets(pid).add) 23 | degrees(pid) += hSet.size 24 | (h, pid) 25 | } 26 | }.cache() 27 | 28 | vRDD = hRDD.flatMap(h => HyperUtils.iteratorFromHString(h._1)) 29 | .distinct(k).map{v => 30 | val pid = Random.nextInt(k) 31 | (v, pid) 32 | }.cache() 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/VertexPartitioner.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.partition 2 | 3 | import org.apache.spark.Partitioner 4 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashMap 5 | import org.apache.spark.hyperx.{PartitionId, VertexId} 6 | 7 | 8 | class VertexPartitioner (val k: Int, val map: HyperXOpenHashMap[VertexId, PartitionId]) 9 | extends Partitioner { 10 | 11 | override def numPartitions: Int = k 12 | 13 | override def getPartition(key: Any): Int = { 14 | key match { 15 | case vid: VertexId => 16 | map(vid) 17 | case _ => 18 | null.asInstanceOf[Int] 19 | } 20 | } 21 | } 22 | 23 | object VertexPartitioner { 24 | def apply(k: Int, map: Iterator[(VertexId, PartitionId)]) = { 25 | val partitioner = new VertexPartitioner(k, new HyperXOpenHashMap[VertexId, PartitionId]()) 26 | map.foreach{v => partitioner.map.update(v._1, v._2)} 27 | partitioner 28 | } 29 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/partition/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | /** 4 | * Collection of partitioning heuristics 5 | */ 6 | package object partition { 7 | var costDegree = 5.0 8 | var costReplica = 10.0 9 | var costDemand = 1.0 10 | var norm = 2 11 | 12 | var effectiveSrc = 0.0 13 | var effectiveDst = 1.0 14 | 15 | private[partition] val searchEpsilonFraction = 0.01 16 | } 17 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/BytecodeUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.util 19 | 20 | import java.io.{ByteArrayInputStream, ByteArrayOutputStream} 21 | 22 | import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ 23 | import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} 24 | import org.apache.spark.util.Utils 25 | 26 | import scala.collection.mutable 27 | import scala.language.existentials 28 | 29 | 30 | /** 31 | * Includes an utility function to test whether a function accesses a 32 | * specific attribute 33 | * of an object. 34 | * 35 | * Forked from GraphX 2.10, modified by Jin Huang 36 | */ 37 | private[hyperx] object BytecodeUtils { 38 | 39 | /** 40 | * Test whether the given closure invokes the specified method in the 41 | * specified class. 42 | */ 43 | def invokedMethod(closure: AnyRef, targetClass: Class[_], 44 | targetMethod: String): Boolean = { 45 | if (_invokedMethod(closure.getClass, "apply", targetClass, 46 | targetMethod)) { 47 | true 48 | } else { 49 | // look at closures enclosed in this closure 50 | for (f <- closure.getClass.getDeclaredFields 51 | if f.getType.getName.startsWith("scala.Function")) { 52 | f.setAccessible(true) 53 | if (invokedMethod(f.get(closure), targetClass, targetMethod)) { 54 | return true 55 | } 56 | } 57 | false 58 | } 59 | } 60 | 61 | private def _invokedMethod(cls: Class[_], method: String, 62 | targetClass: Class[_], targetMethod: String): 63 | Boolean = { 64 | 65 | val seen = new mutable.HashSet[(Class[_], String)] 66 | var stack = List[(Class[_], String)]((cls, method)) 67 | 68 | while (stack.nonEmpty) { 69 | val (c, m) = stack.head 70 | stack = stack.tail 71 | seen.add((c, m)) 72 | val finder = new MethodInvocationFinder(c.getName, m) 73 | getClassReader(c).accept(finder, 0) 74 | for (classMethod <- finder.methodsInvoked) { 75 | // println(classMethod) 76 | if (classMethod._1 == targetClass && classMethod._2 == 77 | targetMethod) { 78 | return true 79 | } else if (!seen.contains(classMethod)) { 80 | stack = classMethod :: stack 81 | } 82 | } 83 | } 84 | false 85 | } 86 | 87 | /** 88 | * Get an ASM class reader for a given class from the JAR that loaded it. 89 | */ 90 | private def getClassReader(cls: Class[_]): ClassReader = { 91 | // Copy data over, before delegating to ClassReader - else we can run 92 | // out of open file handles. 93 | val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" 94 | val resourceStream = cls.getResourceAsStream(className) 95 | // todo: Fixme - continuing with earlier behavior ... 96 | if (resourceStream == null) return new ClassReader(resourceStream) 97 | 98 | val baos = new ByteArrayOutputStream(128) 99 | Utils.copyStream(resourceStream, baos, closeStreams = true) 100 | new ClassReader(new ByteArrayInputStream(baos.toByteArray)) 101 | } 102 | 103 | /** 104 | * Given the class name, return whether we should look into the class or 105 | * not. This is used to 106 | * skip examing a large quantity of Java or Scala classes that we know 107 | * for sure wouldn't access 108 | * the closures. Note that the class name is expected in ASM style (i.e. 109 | * use "/" instead of "."). 110 | */ 111 | private def skipClass(className: String): Boolean = { 112 | val c = className 113 | c.startsWith("java/") || c.startsWith("scala/") || c.startsWith("javax/") 114 | } 115 | 116 | /** 117 | * Find the set of methods invoked by the specified method in the 118 | * specified class. 119 | * For example, after running the visitor, 120 | * MethodInvocationFinder("spark/graph/Foo", "test") 121 | * its methodsInvoked variable will contain the set of methods invoked 122 | * directly by 123 | * Foo.test(). Interface invocations are not returned as part of the 124 | * result set because we cannot 125 | * determine the actual metod invoked by inspecting the bytecode. 126 | */ 127 | private class MethodInvocationFinder(className: String, methodName: String) 128 | extends ClassVisitor(ASM4) { 129 | 130 | val methodsInvoked = new mutable.HashSet[(Class[_], String)] 131 | 132 | override def visitMethod(access: Int, name: String, desc: String, 133 | sig: String, exceptions: Array[String]): 134 | MethodVisitor = { 135 | if (name == methodName) { 136 | new MethodVisitor(ASM4) { 137 | override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { 138 | if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { 139 | if (!skipClass(owner)) { 140 | methodsInvoked.add((Class.forName(owner.replace("/", ".")), name)) 141 | } 142 | } 143 | } 144 | } 145 | } else { 146 | null 147 | } 148 | } 149 | } 150 | 151 | } 152 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/HyperUtils.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.util 2 | 3 | import org.apache.spark.hyperx.util.collection.{HyperXLongBitSet, HyperXOpenHashMap} 4 | import org.apache.spark.hyperx.{HyperAttr, PartitionId, VertexId, VertexSet} 5 | import org.apache.spark.util.collection.{BitSet, OpenHashSet} 6 | 7 | import scala.collection.mutable 8 | import scala.reflect.ClassTag 9 | 10 | /** Forked from GraphX 2.10, modified by Jin Huang */ 11 | object HyperUtils { 12 | 13 | /** 14 | * Determine whether two vertex sets equal to each other 15 | */ 16 | def is(a: VertexSet, b: VertexSet): Boolean = { 17 | if (a.size != b.size) false 18 | a.iterator.map(b.contains).reduce(_ && _) 19 | } 20 | 21 | // def is(a: HyperXOpenHashSet[VertexId], b: VertexSet): 22 | // Boolean = { 23 | // if (a.size != b.size) false 24 | // else a.iterator.map(v => b.contains(v)).reduce(_ && _) 25 | // } 26 | 27 | def toString(t: (VertexSet, VertexSet)): String = { 28 | toString(t._1) + ";" + toString(t._2) 29 | } 30 | 31 | def toString(s: VertexSet): String = { 32 | s.iterator.toArray.sortBy(v => v).map(v => v.toString) reduce (_ + " " + _) 33 | } 34 | 35 | def iteratorFromString(s: String): Iterator[VertexId] = { 36 | // s.split(" ").map(_.toInt).iterator 37 | // s.split(" ").map(_.toLong).iterator 38 | setFromString(s).iterator 39 | } 40 | 41 | def setFromString(s: String): VertexSet = { 42 | setFromString(s, " ") 43 | } 44 | 45 | def setFromString(s: String, separator: String): VertexSet = { 46 | // val inputArray = s.trim.split(separator).map(_.toInt).toVector 47 | val inputArray = s.trim.split(separator).map(_.toLong).toVector 48 | val set = new VertexSet() 49 | inputArray.foreach(set.add) 50 | set 51 | } 52 | 53 | def setFromHString(s: String): Set[VertexId] = { 54 | iteratorFromHString(s).toSet 55 | } 56 | 57 | def hyperedgeFromHString(s: String): (VertexSet, VertexSet) = { 58 | val array = s.split(";") 59 | (setFromString(array(0)), setFromString(array(1))) 60 | } 61 | 62 | def bitFromString(s: String): (HyperXLongBitSet, HyperXLongBitSet) = { 63 | val array = s.split(";") 64 | val srcArray = array(0).trim.split(" ").map(_.toLong).toVector 65 | val srcSet = new HyperXLongBitSet() 66 | srcArray.foreach(srcSet.add) 67 | val dstArray = array(1).trim.split(" ").map(_.toLong).toVector 68 | val dstSet = new HyperXLongBitSet() 69 | dstArray.foreach(dstSet.add) 70 | (srcSet, dstSet) 71 | } 72 | 73 | def countDegreeFromHString(s: String): Int = { 74 | val h = hyperedgeFromHString(s) 75 | h._1.size + h._2.size 76 | } 77 | 78 | def countDetailDegreeFromHString(s: String): (Int, Int) = { 79 | val h = hyperedgeFromHString(s) 80 | (h._1.size, h._2.size) 81 | } 82 | 83 | def iteratorFromHString(s: String): Iterator[VertexId] = { 84 | // s.split(";").flatMap(vs => vs.trim.split(" ").map(_.toInt)).iterator 85 | // s.split(";").flatMap(vs => vs.trim.split(" ").map(_.toLong)).iterator 86 | val h = hyperedgeFromHString(s) 87 | h._1.iterator ++ h._2.iterator 88 | } 89 | 90 | def pidFromPartitionedString(s: String): PartitionId = { 91 | val array = s.split(";") 92 | array(0).trim.toInt 93 | } 94 | 95 | def iteratorFromPartitionedString(s: String): Iterator[VertexId] = { 96 | val array = s.split(";") 97 | (array(1).trim.split(" ").map(_.toLong) ++ 98 | array(2).trim.split(" ").map(_.toLong)).iterator 99 | } 100 | 101 | def hStringFromPartitionedString(s: String): String = { 102 | val array = s.split(";") 103 | array(1).trim + ";" + array(2).trim 104 | } 105 | 106 | def pairFromPartitionedString(s: String): (PartitionId, String) = { 107 | val array = s.split(";") 108 | Tuple2(array(0).trim.toInt, array(1) + ";" + array(2)) 109 | } 110 | 111 | /** 112 | * Union the two vertex sets 113 | */ 114 | def union(a: VertexSet, b: VertexSet): VertexSet = { 115 | val set = new VertexSet 116 | (a.iterator ++ b.iterator).foreach(set.add) 117 | set 118 | } 119 | 120 | 121 | /** 122 | * Union the two sets stored as two arrays, assuming elements are unique 123 | * in two sets 124 | */ 125 | def union[V: ClassTag](a: Array[V], b: Array[V]): Array[V] = { 126 | val set = new OpenHashSet[V]() 127 | (a ++ b).foreach(v => set.add(v)) 128 | set.iterator.toArray 129 | } 130 | 131 | /** 132 | * Count the number of intersecting elements in the two sets, 133 | * where one of the set is stored in an array 134 | */ 135 | def countIntersection(a: VertexSet, b: Array[VertexId]): Int = { 136 | b.map(e => if(a.contains(e)) 1 else 0).sum 137 | } 138 | 139 | /** 140 | * Count the number of intersecting elements in the two sets, 141 | * where one of the set is stored in an [[OpenHashSet]] 142 | */ 143 | def countIntersection(a: VertexSet, b: OpenHashSet[VertexId]): Int = { 144 | a.iterator.map(v => if (b.contains(v)) 1 else 0).sum 145 | } 146 | 147 | def countIntersect(a: OpenHashSet[VertexId], b: OpenHashSet[VertexId]): Int = { 148 | a.iterator.count(b.contains) 149 | } 150 | 151 | def countIntersect(a: Set[VertexId], b: mutable.HashSet[VertexId]): Int = { 152 | a.iterator.count(b.contains) 153 | } 154 | 155 | def countIntersect(a: Set[VertexId], b: OpenHashSet[VertexId]): Int = { 156 | a.iterator.count(b.contains) 157 | } 158 | 159 | /** 160 | * Count the elements in a given set 161 | */ 162 | def count(id: VertexId, set: VertexSet): (VertexId, Int) = (id, set.size) 163 | 164 | /** 165 | * Compare the size of two vertex sets, return negative if the first is 166 | * smaller than the second, vice versa 167 | */ 168 | def compare(a: VertexSet, b: VertexSet): Int = a.size - b.size 169 | 170 | /** 171 | * Create a new identifier-attribute map for a given vertex identifier set 172 | */ 173 | def init[VD: ClassTag](vids: VertexSet): HyperAttr[VD] = { 174 | val attr = new HyperXOpenHashMap[VertexId, VD]() 175 | val it = vids.iterator 176 | while (it.hasNext) { 177 | attr.update(it.next(), null.asInstanceOf[VD]) 178 | } 179 | attr 180 | } 181 | 182 | def toBitSet(set: OpenHashSet[Int]) = { 183 | val bitset = new BitSet(set.iterator.max + 1) 184 | set.iterator.foreach(bitset.set) 185 | bitset 186 | } 187 | 188 | 189 | /** 190 | * Return a string representation for a set of vertex 191 | * identifier-attribute pairs 192 | * @param attr the identifier-attribute map 193 | * @param sep the separator string between each entry in the map 194 | * @tparam VD the type of the vertex attribute 195 | * @return the string representation 196 | */ 197 | def mkString[VD](attr: HyperAttr[VD], sep: String): String = { 198 | val buf = new StringBuilder 199 | val it = attr.iterator 200 | while (it.hasNext) { 201 | it.next().toString().addString(buf) 202 | sep.addString(buf) 203 | } 204 | buf.toString() 205 | } 206 | 207 | /** 208 | * Return a [[VertexSet]] for the elements in the given array 209 | */ 210 | def arrayToSet(array: Array[VertexId]): VertexSet = { 211 | val set = new VertexSet() 212 | array.foreach(set.add) 213 | set 214 | } 215 | 216 | /** 217 | * Return a [[mutable.HashSet]] for the elements in the given array 218 | */ 219 | def arrayToHashSet[V](array: Array[V]): mutable.HashSet[V] = { 220 | val set = new mutable.HashSet[V]() 221 | array.foreach(e => set.add(e)) 222 | set 223 | } 224 | 225 | /** 226 | * Return a [[mutable.HashMap]] for the elements in the given array, 227 | * using the value to record the frequency of the 228 | * elements in the array 229 | */ 230 | def arrayToHashMap[K](array: Array[K]): mutable.HashMap[K, Int] = { 231 | array.map(e => mutable.HashMap[K, Int]((e, 1))).reduce(concatenate[K]) 232 | } 233 | 234 | /** 235 | * Concatenate two [[mutable.HashMap]] by adding the unprecedented 236 | * elements and summing the frequencies for the 237 | * existing elements 238 | */ 239 | def concatenate[K](a: mutable.HashMap[K, Int], b: mutable.HashMap[K, Int]) 240 | : mutable.HashMap[K, Int] = { 241 | a.foreach(e => if (b.contains(e._1)) b.update(e._1, b(e._1) + e._2) 242 | else b.update(e._1, e._2)) 243 | b 244 | } 245 | 246 | /** 247 | * Return an array for the VertexIds stored in the given VertexSet 248 | */ 249 | def setToArray(s: VertexSet): Array[VertexId] = { 250 | s.iterator.toArray 251 | } 252 | 253 | def divide(dividend: Double, map: HyperAttr[Int]): HyperAttr[Double] = { 254 | val attr = new HyperXOpenHashMap[VertexId, Double]() 255 | map.foreach(each => attr.update(each._1, if (each._2 > 0) dividend / each._2 else 0)) 256 | attr 257 | } 258 | 259 | 260 | def printMemory(): String = { 261 | runtime.freeMemory() / 1000.0 / 1000.0 + "; " + 262 | runtime.totalMemory() / 1000.0/ 1000.0 + "; " + 263 | runtime.maxMemory() / 1000.0 / 1000.0 264 | } 265 | 266 | private[hyperx] def avg(array: Array[Int]): Double = 267 | array.sum * 1.0 / array.length 268 | 269 | private[hyperx] def dvt(array: Array[Int]): Double = { 270 | val average = avg(array) 271 | val sum = array.map(e => Math.pow(Math.abs(e - average), 2)).sum 272 | Math.pow(sum, 1.0 / 2) 273 | } 274 | 275 | private[hyperx] def effectiveCount(srcCount: Int, dstCount: Int): Int = { 276 | // (17 + 0.125 * dstCount + 65 + 0.875 * srcCount).toInt] 277 | (srcCount * 2.5 + dstCount * 1.0).toInt 278 | } 279 | 280 | private val runtime = Runtime.getRuntime 281 | } 282 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/collection/HyperXLongBitSet.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx.util.collection 2 | 3 | import org.apache.spark.util.collection.BitSet 4 | 5 | class HyperXLongBitSet extends Iterable[Long] with Serializable { 6 | 7 | def add(value: Long): Unit ={ 8 | setValue(value, flag = true) 9 | } 10 | 11 | def remove(value: Long): Unit = { 12 | setValue(value, flag = false) 13 | } 14 | 15 | def contains(value: Long): Boolean = { 16 | getValue(value) 17 | } 18 | 19 | override def iterator: Iterator[Long] = { 20 | sets.flatMap{entry => 21 | val bitset = entry._2 22 | val baseIndex = entry._1 << valuebits 23 | bitset.iterator.map(_.toLong + baseIndex) 24 | }.iterator 25 | } 26 | 27 | private val valuebits = 16 28 | private val valuemask = (1 << valuebits) - 1 29 | 30 | private[hyperx] val sets = new HyperXOpenHashMap[Long, BitSet](valuebits) 31 | 32 | private def getSetIndex(index: Long): Long = { 33 | index >> valuebits 34 | } 35 | 36 | private def getPos(index: Long): Int = { 37 | (index & valuemask).toInt 38 | } 39 | 40 | private def bitset(index: Long): BitSet = { 41 | val setIndex = getSetIndex(index) 42 | var set = sets(setIndex) 43 | set = set match { 44 | case existing: BitSet => 45 | existing 46 | case null => 47 | // val newSet = new BitSet(1024) 48 | val newSet = new BitSet(1 << valuebits) 49 | sets.update(setIndex, newSet) 50 | newSet 51 | } 52 | set 53 | } 54 | 55 | private def setValue(index: Long, flag: Boolean): Unit ={ 56 | if (flag) { 57 | // growIfNeeded(index) 58 | bitset(index).set(getPos(index)) 59 | } 60 | else { 61 | val set = sets(getSetIndex(index)) 62 | set match { 63 | case existing: BitSet => 64 | val pos = getPos(index) 65 | if (pos < existing.capacity) { 66 | existing.unset(getPos(index)) 67 | } 68 | } 69 | } 70 | } 71 | 72 | private def getValue(index: Long): Boolean = { 73 | val bitset = sets(getSetIndex(index)) 74 | bitset != null && bitset.get(getPos(index)) 75 | } 76 | 77 | // private def growIfNeeded(index: Long): Unit = { 78 | // val pos = getPos(index) 79 | // val set = bitset(index) 80 | // if (set.capacity <= pos) { 81 | // val newSet = new BitSet(pos + 1) 82 | // set.iterator.foreach(newSet.set) 83 | // sets.update(getSetIndex(index), newSet) 84 | // } 85 | // } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/collection/HyperXOpenHashMap.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.util.collection 19 | 20 | import scala.reflect._ 21 | 22 | /** 23 | * A fast hash map implementation for primitive, non-null keys. This hash map 24 | * supports insertions and updates, but not deletions. This map is about an 25 | * order of magnitude faster than java.util.HashMap, while using much less 26 | * space overhead. 27 | * 28 | * Under the hood, it uses our OpenHashSet implementation. 29 | * 30 | * There is a bug in the original Spark 1.1.0 implementation: 31 | * If there are more than 11744053 pairs stored, the update method will 32 | * miserably fail to set the correct value for the key. Also, the update of the 33 | * value on an existing key may or may not result to correct set. 34 | * 35 | * The bug is caused by wrong pos calculating for a given key: the pos is 36 | * calculated before rehashing the underlying key set, and the value assigned to 37 | * that may overwrite the value of an overflow key. To fix the bug, we first 38 | * add the key, and rehash accordingly; then the pos is calculated based on the 39 | * new key set and the value is assigned. 40 | * 41 | */ 42 | private[hyperx] 43 | class HyperXOpenHashMap[@specialized(Long, Int) K: ClassTag, 44 | @specialized(Long, Int, Double) V: ClassTag](val keySet:HyperXOpenHashSet[K], 45 | var _values: Array[V]) 46 | extends Iterable[(K, V)] 47 | with Serializable { 48 | 49 | // The following member variables are declared as protected instead of 50 | // private for the specialization to work (specialized class extends the 51 | // unspecialized one and needs access to the "private" variables). 52 | // They also should have been val's. We use var's because there is a Scala 53 | // compiler bug that would throw illegal access error at runtime if they are 54 | // declared as val's. 55 | protected var grow = (newCapacity: Int) => { 56 | _oldValues = _values 57 | _values = new Array[V](newCapacity) 58 | } 59 | protected var move = (oldPos: Int, newPos: Int) => { 60 | _values(newPos) = _oldValues(oldPos) 61 | } 62 | private var _oldValues: Array[V] = null 63 | 64 | // require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int]) 65 | 66 | /** 67 | * Allocate an OpenHashMap with a fixed initial capacity 68 | */ 69 | def this(initialCapacity: Int) = 70 | this(new HyperXOpenHashSet[K](initialCapacity), 71 | new Array[V](initialCapacity)) 72 | 73 | /** 74 | * Allocate an OpenHashMap with a fixed initial capacity 75 | */ 76 | def this(keySet: HyperXOpenHashSet[K]) = this(keySet, 77 | new Array[V](keySet.capacity)) 78 | 79 | override def size = keySet.size 80 | 81 | /** Get the value for a given key */ 82 | def apply(k: K): V = { 83 | val pos = keySet.getPos(k) 84 | // _values(pos) 85 | // fix array index out of bound 86 | if (pos >= 0 && _values.size > pos) { 87 | _values(pos) 88 | } 89 | else null.asInstanceOf[V] 90 | } 91 | 92 | // def nth(n: Int): V = { 93 | // if (n >= keySet.size) { 94 | // null.asInstanceOf[V] 95 | // } 96 | // else { 97 | // var pos = keySet.nextPos(0) 98 | // (0 until n).foreach { i => pos = keySet.nextPos(pos + 1)} 99 | // _values(pos) 100 | // } 101 | // } 102 | 103 | def nextPos(pos: Int) = { 104 | keySet.nextPos(pos) 105 | } 106 | 107 | /** Get the value for a given key, or returns elseValue if it doesn't 108 | * exist. */ 109 | def getOrElse(k: K, elseValue: V): V = { 110 | val pos = keySet.getPos(k) 111 | if (pos >= 0) _values(pos) else elseValue 112 | } 113 | 114 | /** Set the value for a key */ 115 | def setMerge(k: K, v: V, mergeF: (V, V) => V) { 116 | // val pos = keySet.addWithoutResize(k) 117 | keySet.addWithoutResize(k) 118 | keySet.rehashIfNeeded(k, grow, move) 119 | val pos = keySet.getPos(k) 120 | val ind = pos & HyperXOpenHashSet.POSITION_MASK 121 | if ((pos & HyperXOpenHashSet.NONEXISTENCE_MASK) != 0) { 122 | // if first add 123 | _values(ind) = v 124 | } else { 125 | _values(ind) = mergeF(_values(ind), v) 126 | } 127 | // keySet.rehashIfNeeded(k, grow, move) 128 | _oldValues = null 129 | } 130 | 131 | 132 | /** 133 | * If the key doesn't exist yet in the hash map, set its value to 134 | * defaultValue; otherwise, set its value to mergeValue(oldValue). 135 | * 136 | * @return the newly updated value. 137 | */ 138 | def changeValue(k: K, defaultValue: => V, mergeValue: (V) => V): V = { 139 | if (!keySet.contains(k)) { 140 | keySet.addWithoutResize(k) 141 | keySet.rehashIfNeeded(k, grow, move) 142 | val pos = keySet.getPos(k) 143 | _values(pos) = defaultValue 144 | _values(pos) 145 | } else { 146 | val pos = keySet.getPos(k) 147 | _values(pos) = mergeValue(_values(pos)) 148 | _values(pos) 149 | } 150 | } 151 | 152 | def ++(other: HyperXOpenHashMap[K, V]): HyperXOpenHashMap[K, V] = { 153 | val merged = new HyperXOpenHashMap[K, V]() 154 | val it: Iterator[(K, V)] = this.iterator ++ other.iterator 155 | while (it.hasNext) { 156 | val cur = it.next() 157 | merged.update(cur._1, cur._2) 158 | } 159 | merged 160 | } 161 | 162 | /** 163 | * Allocate an OpenHashMap with a default initial capacity, providing a true 164 | * no-argument constructor. 165 | */ 166 | def this() = this(64) 167 | 168 | /** Set the value for a key */ 169 | def update(k: K, v: V) { 170 | // fix the wrong pos set 171 | keySet.addWithoutResize(k) 172 | keySet.rehashIfNeeded(k, grow, move) 173 | _oldValues = null 174 | val pos = keySet.getPos(k) 175 | _values(pos) = v 176 | } 177 | 178 | override def iterator = new Iterator[(K, V)] { 179 | var pos = 0 180 | var nextPair: (K, V) = computeNextPair() 181 | 182 | /** Get the next value we should return from next(), 183 | * or null if we're finished iterating */ 184 | def computeNextPair(): (K, V) = { 185 | pos = keySet.nextPos(pos) 186 | if (pos >= 0) { 187 | val ret = (keySet.getValue(pos), _values(pos)) 188 | pos += 1 189 | ret 190 | } else { 191 | null 192 | } 193 | } 194 | 195 | def hasNext = nextPair != null 196 | 197 | def next() = { 198 | val pair = nextPair 199 | nextPair = computeNextPair() 200 | pair 201 | } 202 | } 203 | 204 | def hasKey(k: K): Boolean = keySet.contains(k) 205 | 206 | def clear(): Unit = { 207 | keySet.clear() 208 | _values = null 209 | } 210 | 211 | def toArrays: (Array[K], Array[V]) = { 212 | val keyAry = new HyperXPrimitiveVector[K]() 213 | val valAry = new HyperXPrimitiveVector[V]() 214 | iterator.foreach{pair=> 215 | keyAry += pair._1 216 | valAry += pair._2 217 | } 218 | (keyAry.trim().array, valAry.trim().array) 219 | } 220 | 221 | def mapOn(f: (V) => V): HyperXOpenHashMap[K, V] = { 222 | this.iterator.foreach{pair=> 223 | this.update(pair._1, f(pair._2)) 224 | } 225 | this 226 | } 227 | 228 | def mapOn(f: (K, V) => V): HyperXOpenHashMap[K, V] = { 229 | this.iterator.foreach{pair=> 230 | this.update(pair._1, f(pair._1, pair._2)) 231 | } 232 | this 233 | } 234 | } 235 | 236 | private[hyperx] 237 | object HyperXOpenHashMap { 238 | def apply[K: ClassTag, V: ClassTag](iter: Iterator[(K, V)]): HyperXOpenHashMap[K, V] = { 239 | val map = new HyperXOpenHashMap[K, V]() 240 | iter.foreach(pair => map.update(pair._1, pair._2)) 241 | map 242 | } 243 | } 244 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/collection/HyperXOpenHashSet.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.util.collection 19 | 20 | import com.google.common.hash.Hashing 21 | import org.apache.spark.util.collection.BitSet 22 | 23 | import scala.reflect._ 24 | 25 | /** 26 | * A simple, fast hash set optimized for non-null insertion-only use case, where keys are never 27 | * removed. 28 | * 29 | * The underlying implementation uses Scala compiler's specialization to generate optimized 30 | * storage for two primitive types (Long and Int). It is much faster than Java's standard HashSet 31 | * while incurring much less memory overhead. This can serve as building blocks for higher level 32 | * data structures such as an optimized HashMap. 33 | * 34 | * This OpenHashSet is designed to serve as building blocks for higher level data structures 35 | * such as an optimized hash map. Compared with standard hash set implementations, this class 36 | * provides its various callbacks interfaces (e.g. allocateFunc, moveFunc) and interfaces to 37 | * retrieve the position of a key in the underlying array. 38 | * 39 | * It uses quadratic probing with a power-of-2 hash table size, which is guaranteed 40 | * to explore all spaces for each key (see http://en.wikipedia.org/wiki/Quadratic_probing). 41 | */ 42 | private[spark] 43 | class HyperXOpenHashSet[@specialized(Long, Int) T: ClassTag]( 44 | initialCapacity: Int, 45 | loadFactor: Double) 46 | extends Serializable { 47 | 48 | require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") 49 | require(initialCapacity >= 1, "Invalid initial capacity") 50 | require(loadFactor < 1.0, "Load factor must be less than 1.0") 51 | require(loadFactor > 0.0, "Load factor must be greater than 0.0") 52 | 53 | import org.apache.spark.hyperx.util.collection.HyperXOpenHashSet._ 54 | 55 | def this(initialCapacity: Int) = this(initialCapacity, 0.7) 56 | 57 | def this() = this(64) 58 | 59 | // The following member variables are declared as protected instead of private for the 60 | // specialization to work (specialized class extends the non-specialized one and needs access 61 | // to the "private" variables). 62 | 63 | protected val hasher: Hasher[T] = { 64 | // It would've been more natural to write the following using pattern matching. But Scala 2.9.x 65 | // compiler has a bug when specialization is used together with this pattern matching, and 66 | // throws: 67 | // scala.tools.nsc.symtab.Types$TypeError: type mismatch; 68 | // found : scala.reflect.AnyValManifest[Long] 69 | // required: scala.reflect.ClassTag[Int] 70 | // at scala.tools.nsc.typechecker.Contexts$Context.error(Contexts.scala:298) 71 | // at scala.tools.nsc.typechecker.Infer$Inferencer.error(Infer.scala:207) 72 | // ... 73 | val mt = classTag[T] 74 | if (mt == ClassTag.Long) { 75 | (new LongHasher).asInstanceOf[Hasher[T]] 76 | } else if (mt == ClassTag.Int) { 77 | (new IntHasher).asInstanceOf[Hasher[T]] 78 | } else { 79 | new Hasher[T] 80 | } 81 | } 82 | 83 | protected var _capacity = nextPowerOf2(initialCapacity) 84 | protected var _mask = _capacity - 1 85 | protected var _size = 0 86 | protected var _growThreshold = (loadFactor * _capacity).toInt 87 | 88 | protected var _bitset = new BitSet(_capacity) 89 | 90 | def getBitSet = _bitset 91 | 92 | // Init of the array in constructor (instead of in declaration) to work around a Scala compiler 93 | // specialization bug that would generate two arrays (one for Object and one for specialized T). 94 | protected var _data: Array[T] = _ 95 | _data = new Array[T](_capacity) 96 | 97 | /** Number of elements in the set. */ 98 | def size: Int = _size 99 | 100 | /** The capacity of the set (i.e. size of the underlying array). */ 101 | def capacity: Int = _capacity 102 | 103 | /** Return true if this set contains the specified element. */ 104 | def contains(k: T): Boolean = getPos(k) != INVALID_POS 105 | 106 | /** 107 | * Add an element to the set. If the set is over capacity after the insertion, grow the set 108 | * and rehash all elements. 109 | */ 110 | def add(k: T) { 111 | addWithoutResize(k) 112 | rehashIfNeeded(k, grow, move) 113 | } 114 | 115 | /** 116 | * Add an element to the set. This one differs from add in that it doesn't trigger rehashing. 117 | * The caller is responsible for calling rehashIfNeeded. 118 | * 119 | * Use (retval & POSITION_MASK) to get the actual position, and 120 | * (retval & NONEXISTENCE_MASK) == 0 for prior existence. 121 | * 122 | * @return The position where the key is placed, plus the highest order bit is set if the key 123 | * does not exists previously. 124 | */ 125 | def addWithoutResize(k: T): Int = { 126 | var pos = hashcode(hasher.hash(k)) & _mask 127 | var i = 1 128 | while (true) { 129 | if (!_bitset.get(pos)) { 130 | // This is a new key. 131 | _data(pos) = k 132 | _bitset.set(pos) 133 | _size += 1 134 | return pos | NONEXISTENCE_MASK 135 | } else if (_data(pos) == k) { 136 | // Found an existing key. 137 | return pos 138 | } else { 139 | val delta = i 140 | pos = (pos + delta) & _mask 141 | i += 1 142 | } 143 | } 144 | // Never reached here 145 | assert(INVALID_POS != INVALID_POS) 146 | INVALID_POS 147 | } 148 | 149 | /** 150 | * Rehash the set if it is overloaded. 151 | * @param k A parameter unused in the function, but to force the Scala compiler to specialize 152 | * this method. 153 | * @param allocateFunc Callback invoked when we are allocating a new, larger array. 154 | * @param moveFunc Callback invoked when we move the key from one position (in the old data array) 155 | * to a new position (in the new data array). 156 | */ 157 | def rehashIfNeeded(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) { 158 | if (_size > _growThreshold) { 159 | rehash(k, allocateFunc, moveFunc) 160 | } 161 | } 162 | 163 | /** 164 | * Return the position of the element in the underlying array, or INVALID_POS if it is not found. 165 | */ 166 | def getPos(k: T): Int = { 167 | var pos = hashcode(hasher.hash(k)) & _mask 168 | var i = 1 169 | val maxProbe = _data.size 170 | while (i < maxProbe) { 171 | if (!_bitset.get(pos)) { 172 | return INVALID_POS 173 | } else if (k == _data(pos)) { 174 | return pos 175 | } else { 176 | val delta = i 177 | pos = (pos + delta) & _mask 178 | i += 1 179 | } 180 | } 181 | // Never reached here 182 | INVALID_POS 183 | } 184 | 185 | /** Return the value at the specified position. */ 186 | def getValue(pos: Int): T = _data(pos) 187 | 188 | def iterator = new Iterator[T] { 189 | var pos = nextPos(0) 190 | override def hasNext: Boolean = pos != INVALID_POS 191 | override def next(): T = { 192 | val tmp = getValue(pos) 193 | pos = nextPos(pos + 1) 194 | tmp 195 | } 196 | } 197 | 198 | /** Return the value at the specified position. */ 199 | def getValueSafe(pos: Int): T = { 200 | assert(_bitset.get(pos)) 201 | _data(pos) 202 | } 203 | 204 | /** 205 | * Return the next position with an element stored, starting from the given position inclusively. 206 | */ 207 | def nextPos(fromPos: Int): Int = _bitset.nextSetBit(fromPos) 208 | 209 | def clear(): Unit = { 210 | _data = null 211 | _bitset = null 212 | } 213 | 214 | /** 215 | * Double the table's size and re-hash everything. We are not really using k, but it is declared 216 | * so Scala compiler can specialize this method (which leads to calling the specialized version 217 | * of putInto). 218 | * 219 | * @param k A parameter unused in the function, but to force the Scala compiler to specialize 220 | * this method. 221 | * @param allocateFunc Callback invoked when we are allocating a new, larger array. 222 | * @param moveFunc Callback invoked when we move the key from one position (in the old data array) 223 | * to a new position (in the new data array). 224 | */ 225 | private def rehash(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) { 226 | val newCapacity = _capacity * 2 227 | allocateFunc(newCapacity) 228 | val newBitset = new BitSet(newCapacity) 229 | val newData = new Array[T](newCapacity) 230 | val newMask = newCapacity - 1 231 | 232 | var oldPos = 0 233 | while (oldPos < capacity) { 234 | if (_bitset.get(oldPos)) { 235 | val key = _data(oldPos) 236 | var newPos = hashcode(hasher.hash(key)) & newMask 237 | var i = 1 238 | var keepGoing = true 239 | // No need to check for equality here when we insert so this has one less if branch than 240 | // the similar code path in addWithoutResize. 241 | while (keepGoing) { 242 | if (!newBitset.get(newPos)) { 243 | // Inserting the key at newPos 244 | newData(newPos) = key 245 | newBitset.set(newPos) 246 | moveFunc(oldPos, newPos) 247 | keepGoing = false 248 | } else { 249 | val delta = i 250 | newPos = (newPos + delta) & newMask 251 | i += 1 252 | } 253 | } 254 | } 255 | oldPos += 1 256 | } 257 | 258 | _bitset = newBitset 259 | _data = newData 260 | _capacity = newCapacity 261 | _mask = newMask 262 | _growThreshold = (loadFactor * newCapacity).toInt 263 | } 264 | 265 | /** 266 | * Re-hash a value to deal better with hash functions that don't differ in the lower bits. 267 | */ 268 | private def hashcode(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() 269 | 270 | private def nextPowerOf2(n: Int): Int = { 271 | val highBit = Integer.highestOneBit(n) 272 | if (highBit == n) n else highBit << 1 273 | } 274 | } 275 | 276 | 277 | private[spark] 278 | object HyperXOpenHashSet { 279 | 280 | val INVALID_POS = -1 281 | val NONEXISTENCE_MASK = 0x80000000 282 | val POSITION_MASK = 0xEFFFFFF 283 | 284 | /** 285 | * A set of specialized hash function implementation to avoid boxing hash code computation 286 | * in the specialized implementation of OpenHashSet. 287 | */ 288 | sealed class Hasher[@specialized(Long, Int) T] extends Serializable { 289 | def hash(o: T): Int = o.hashCode() 290 | } 291 | 292 | class LongHasher extends Hasher[Long] { 293 | override def hash(o: Long): Int = (o ^ (o >>> 32)).toInt 294 | } 295 | 296 | class IntHasher extends Hasher[Int] { 297 | override def hash(o: Int): Int = o 298 | } 299 | 300 | private def grow1(newSize: Int) {} 301 | private def move1(oldPos: Int, newPos: Int) { } 302 | 303 | private val grow = grow1 _ 304 | private val move = move1 _ 305 | 306 | // private implicit def toLongBitSet(hash: HyperXOpenHashSet[Long]): HyperXLongBitSet = { 307 | // val set = new HyperXLongBitSet 308 | // hash.iterator.foreach(set.add) 309 | // set 310 | // } 311 | } 312 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/collection/HyperXPrimitiveVector.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.hyperx.util.collection 19 | 20 | import scala.reflect.ClassTag 21 | 22 | /** 23 | * An append-only, non-threadsafe, array-backed vector that is optimized for primitive types. 24 | */ 25 | private[hyperx] 26 | class HyperXPrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: Int = 64) extends Serializable { 27 | private var _numElements = 0 28 | private var _array: Array[V] = _ 29 | 30 | // NB: This must be separate from the declaration, otherwise the specialized parent class 31 | // will get its own array with the same initial size. 32 | _array = new Array[V](initialSize) 33 | 34 | def apply(index: Int): V = { 35 | require(index < _numElements) 36 | _array(index) 37 | } 38 | 39 | def +=(value: V): Unit = { 40 | if (_numElements >= _array.length) { 41 | resize(_array.length * 2) 42 | } 43 | _array(_numElements) = value 44 | _numElements += 1 45 | } 46 | 47 | def capacity: Int = _array.length 48 | 49 | def length: Int = _numElements 50 | 51 | def size: Int = _numElements 52 | 53 | def iterator: Iterator[V] = new Iterator[V] { 54 | var index = 0 55 | override def hasNext: Boolean = index < _numElements 56 | override def next(): V = { 57 | if (!hasNext) { 58 | throw new NoSuchElementException 59 | } 60 | val value = _array(index) 61 | index += 1 62 | value 63 | } 64 | } 65 | 66 | /** Gets the underlying array backing this vector. */ 67 | def array: Array[V] = _array 68 | 69 | /** Trims this vector so that the capacity is equal to the size. */ 70 | def trim(): HyperXPrimitiveVector[V] = resize(size) 71 | 72 | /** Resizes the array, dropping elements if the total length decreases. */ 73 | def resize(newLength: Int): HyperXPrimitiveVector[V] = { 74 | val newArray = new Array[V](newLength) 75 | _array.copyToArray(newArray) 76 | _array = newArray 77 | if (newLength < _numElements) { 78 | _numElements = newLength 79 | } 80 | this 81 | } 82 | } 83 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/hyperx/util/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.hyperx 2 | 3 | /** 4 | * Collection of objects used by HyperX 5 | */ 6 | package object util { 7 | 8 | } 9 | --------------------------------------------------------------------------------