Skip to content

Commit 93f3280

Browse files
author
Jacky Li
committed
create FPTree class
1 parent d110ab2 commit 93f3280

File tree

5 files changed

+480
-117
lines changed

5 files changed

+480
-117
lines changed

mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala

Lines changed: 53 additions & 103 deletions
Original file line numberDiff line numberDiff line change
@@ -17,29 +17,34 @@
1717

1818
package org.apache.spark.mllib.fpm
1919

20+
import scala.collection.mutable.ArrayBuffer
21+
22+
import org.apache.spark.broadcast.Broadcast
2023
import org.apache.spark.Logging
21-
import org.apache.spark.SparkContext._
22-
import org.apache.spark.broadcast._
2324
import 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
*/
3338
class 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
/**

mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,5 +20,5 @@ package org.apache.spark.mllib.fpm
2020
/**
2121
* A FPGrowth Model for FPGrowth, each element is a frequent pattern with count.
2222
*/
23-
class FPGrowthModel (val frequentPattern: Array[(String, Int)]) extends Serializable {
23+
class FPGrowthModel (val frequentPattern: Array[(Array[String], Long)]) extends Serializable {
2424
}

0 commit comments

Comments
 (0)