├── .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 |
--------------------------------------------------------------------------------