1717
1818package org .apache .spark .mllib .fpm
1919
20+ import scala .collection .mutable .ArrayBuffer
21+
22+ import org .apache .spark .broadcast .Broadcast
2023import org .apache .spark .Logging
21- import org .apache .spark .SparkContext ._
22- import org .apache .spark .broadcast ._
2324import org .apache .spark .rdd .RDD
2425
25- import scala . collection . mutable .{ ArrayBuffer , Map }
26+
2627
2728/**
28- * This class implements Parallel FPGrowth algorithm to do frequent pattern matching on input data.
29+ * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data.
2930 * Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an
3031 * independent group of mining tasks. More detail of this algorithm can be found at
31- * http://infolab.stanford.edu/~echang/recsys08-69.pdf
32+ * [[http://dx.doi.org/10.1145/1454008.1454027, PFP ]], and the original FP-growth paper can be found at
33+ * [[http://dx.doi.org/10.1145/335191.335372, FP-growth ]]
34+ *
35+ * @param minSupport the minimal support level of the frequent pattern, any pattern appears more than
36+ * (minSupport * size-of-the-dataset) times will be output
3237 */
3338class FPGrowth private (private var minSupport : Double ) extends Logging with Serializable {
3439
3540 /**
3641 * Constructs a FPGrowth instance with default parameters:
37- * {minSupport: 0.5 }
42+ * {minSupport: 0.3 }
3843 */
39- def this () = this (0.5 )
44+ def this () = this (0.3 )
4045
4146 /**
42- * set the minimal support level, default is 0.5
47+ * set the minimal support level, default is 0.3
4348 * @param minSupport minimal support level
4449 */
4550 def setMinSupport (minSupport : Double ): this .type = {
@@ -49,87 +54,82 @@ class FPGrowth private(private var minSupport: Double) extends Logging with Seri
4954
5055 /**
5156 * Compute a FPGrowth Model that contains frequent pattern result.
52- * @param data input data set
57+ * @param data input data set, each element contains a transaction
5358 * @return FPGrowth Model
5459 */
5560 def run (data : RDD [Array [String ]]): FPGrowthModel = {
56- val model = runAlgorithm(data)
57- model
58- }
59-
60- /**
61- * Implementation of PFP.
62- */
63- private def runAlgorithm (data : RDD [Array [String ]]): FPGrowthModel = {
6461 val count = data.count()
6562 val minCount = minSupport * count
6663 val single = generateSingleItem(data, minCount)
6764 val combinations = generateCombinations(data, minCount, single)
68- new FPGrowthModel (single ++ combinations)
65+ val all = single.map(v => (Array [String ](v._1), v._2)).union(combinations)
66+ new FPGrowthModel (all.collect())
6967 }
7068
7169 /**
7270 * Generate single item pattern by filtering the input data using minimal support level
71+ * @return array of frequent pattern with its count
7372 */
7473 private def generateSingleItem (
7574 data : RDD [Array [String ]],
76- minCount : Double ): Array [(String , Int )] = {
77- data.flatMap(v => v)
78- .map(v => (v, 1 ))
75+ minCount : Double ): RDD [(String , Long )] = {
76+ val single = data.flatMap(v => v.toSet )
77+ .map(v => (v, 1L ))
7978 .reduceByKey(_ + _)
8079 .filter(_._2 >= minCount)
81- .collect()
82- .distinct
83- .sortWith(_._2 > _._2)
80+ .sortBy(_._2)
81+ single
8482 }
8583
8684 /**
87- * Generate combination of items by computing on FPTree,
85+ * Generate combination of frequent pattern by computing on FPTree,
8886 * the computation is done on each FPTree partitions.
87+ * @return array of frequent pattern with its count
8988 */
9089 private def generateCombinations (
9190 data : RDD [Array [String ]],
9291 minCount : Double ,
93- singleItem : Array [(String , Int )]): Array [(String , Int )] = {
94- val single = data.context.broadcast(singleItem)
95- data.flatMap(basket => createFPTree(basket, single))
96- .groupByKey()
97- .flatMap(partition => runFPTree(partition, minCount))
98- .collect()
92+ singleItem : RDD [(String , Long )]): RDD [(Array [String ], Long )] = {
93+ val single = data.context.broadcast(singleItem.collect())
94+ data.flatMap(transaction => createConditionPatternBase(transaction, single))
95+ .aggregateByKey(new FPTree )(
96+ (aggregator, condPattBase) => aggregator.add(condPattBase),
97+ (aggregator1, aggregator2) => aggregator1.merge(aggregator2))
98+ .flatMap(partition => partition._2.mine(minCount, partition._1))
9999 }
100100
101101 /**
102102 * Create FP-Tree partition for the giving basket
103+ * @return an array contains a tuple, whose first element is the single
104+ * item (hash key) and second element is its condition pattern base
103105 */
104- private def createFPTree (
105- basket : Array [String ],
106- singleItem : Broadcast [Array [(String , Int )]]): Array [(String , Array [String ])] = {
106+ private def createConditionPatternBase (
107+ transaction : Array [String ],
108+ singleBC : Broadcast [Array [(String , Long )]]): Array [(String , Array [String ])] = {
107109 var output = ArrayBuffer [(String , Array [String ])]()
108110 var combination = ArrayBuffer [String ]()
109- val single = singleItem.value
110- var items = ArrayBuffer [(String , Int )]()
111-
112- // Filter the basket by single item pattern
113- val iterator = basket.iterator
114- while (iterator.hasNext){
115- val item = iterator.next
116- val opt = single.find(_._1.equals(item))
117- if (opt != None ) {
118- items ++= opt
119- }
120- }
121-
122- // Sort it and create the item combinations
123- val sortedItems = items.sortWith(_._1 > _._1).sortWith(_._2 > _._2).toArray
124- val itemIterator = sortedItems.iterator
111+ var items = ArrayBuffer [(String , Long )]()
112+ val single = singleBC.value
113+ val singleMap = single.toMap
114+
115+ // Filter the basket by single item pattern and sort
116+ // by single item and its count
117+ val candidates = transaction
118+ .filter(singleMap.contains)
119+ .map(item => (item, singleMap(item)))
120+ .sortBy(_._1)
121+ .sortBy(_._2)
122+ .toArray
123+
124+ val itemIterator = candidates.iterator
125125 while (itemIterator.hasNext) {
126126 combination.clear()
127- val item = itemIterator.next
128- val firstNItems = sortedItems .take(sortedItems .indexOf(item))
127+ val item = itemIterator.next()
128+ val firstNItems = candidates .take(candidates .indexOf(item))
129129 if (firstNItems.length > 0 ) {
130130 val iterator = firstNItems.iterator
131131 while (iterator.hasNext) {
132- val elem = iterator.next
132+ val elem = iterator.next()
133133 combination += elem._1
134134 }
135135 output += ((item._1, combination.toArray))
@@ -138,56 +138,6 @@ class FPGrowth private(private var minSupport: Double) extends Logging with Seri
138138 output.toArray
139139 }
140140
141- /**
142- * Generate frequent pattern by walking through the FPTree
143- */
144- private def runFPTree (
145- partition : (String , Iterable [Array [String ]]),
146- minCount : Double ): Array [(String , Int )] = {
147- val key = partition._1
148- val value = partition._2
149- val output = ArrayBuffer [(String , Int )]()
150- val map = Map [String , Int ]()
151-
152- // Walk through the FPTree partition to generate all combinations that satisfy
153- // the minimal support level.
154- var k = 1
155- while (k > 0 ) {
156- map.clear()
157- val iterator = value.iterator
158- while (iterator.hasNext) {
159- val pattern = iterator.next
160- if (pattern.length >= k) {
161- val combination = pattern.toList.combinations(k).toList
162- val itemIterator = combination.iterator
163- while (itemIterator.hasNext){
164- val item = itemIterator.next
165- val list2key : List [String ] = (item :+ key).sortWith(_ > _)
166- val newKey = list2key.mkString(" " )
167- if (map.get(newKey) == None ) {
168- map(newKey) = 1
169- } else {
170- map(newKey) = map.apply(newKey) + 1
171- }
172- }
173- }
174- }
175- var eligible : Array [(String , Int )] = null
176- if (map.size != 0 ) {
177- val candidate = map.filter(_._2 >= minCount)
178- if (candidate.size != 0 ) {
179- eligible = candidate.toArray
180- output ++= eligible
181- }
182- }
183- if ((eligible == null ) || (eligible.length == 0 )) {
184- k = 0
185- } else {
186- k = k + 1
187- }
188- }
189- output.toArray
190- }
191141}
192142
193143/**
0 commit comments