-
Notifications
You must be signed in to change notification settings - Fork 28.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-4001][MLlib] adding parallel FP-Growth algorithm for frequent pattern mining in MLlib #2847
Changes from 8 commits
da2cba7
889b33f
f68a0bd
7b77ad7
03df2b6
eb3e4ca
a6c5081
d110ab2
93f3280
ec21f7d
7e69725
bee3093
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,208 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.mllib.fpm | ||
|
||
import org.apache.spark.Logging | ||
import org.apache.spark.SparkContext._ | ||
import org.apache.spark.broadcast._ | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. try to be explicit on imports |
||
import org.apache.spark.rdd.RDD | ||
|
||
import scala.collection.mutable.{ArrayBuffer, Map} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. order imports into groups: https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide#SparkCodeStyleGuide-Imports For a mutable collection, we can import it directly if there is no immutable collection with the same name, e.g., ArrayBuffer. Otherwise, we only import |
||
|
||
/** | ||
* This class implements Parallel FPGrowth algorithm to do frequent pattern matching on input data. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
* Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an | ||
* independent group of mining tasks. More detail of this algorithm can be found at | ||
* http://infolab.stanford.edu/~echang/recsys08-69.pdf | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's use the permanent link: http://dx.doi.org/10.1145/1454008.1454027 It is nice to cite the FP-Growth paper from Han: http://dx.doi.org/10.1145/335191.335372 |
||
*/ | ||
class FPGrowth private(private var minSupport: Double) extends Logging with Serializable { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Doc minSupport. |
||
|
||
/** | ||
* Constructs a FPGrowth instance with default parameters: | ||
* {minSupport: 0.5} | ||
*/ | ||
def this() = this(0.5) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is it too high? How do you set the default value? |
||
|
||
/** | ||
* set the minimal support level, default is 0.5 | ||
* @param minSupport minimal support level | ||
*/ | ||
def setMinSupport(minSupport: Double): this.type = { | ||
this.minSupport = minSupport | ||
this | ||
} | ||
|
||
/** | ||
* Compute a FPGrowth Model that contains frequent pattern result. | ||
* @param data input data set | ||
* @return FPGrowth Model | ||
*/ | ||
def run(data: RDD[Array[String]]): FPGrowthModel = { | ||
val model = runAlgorithm(data) | ||
model | ||
} | ||
|
||
/** | ||
* Implementation of PFP. | ||
*/ | ||
private def runAlgorithm(data: RDD[Array[String]]): FPGrowthModel = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is not necessary to have |
||
val count = data.count() | ||
val minCount = minSupport * count | ||
val single = generateSingleItem(data, minCount) | ||
val combinations = generateCombinations(data, minCount, single) | ||
new FPGrowthModel(single ++ combinations) | ||
} | ||
|
||
/** | ||
* Generate single item pattern by filtering the input data using minimal support level | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should document at least the return value. It would be best document the input arguments as well. |
||
*/ | ||
private def generateSingleItem( | ||
data: RDD[Array[String]], | ||
minCount: Double): Array[(String, Int)] = { | ||
data.flatMap(v => v) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are we assuming that the items inside the same basket are distinct? If not, this could be changed to |
||
.map(v => (v, 1)) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
.reduceByKey(_ + _) | ||
.filter(_._2 >= minCount) | ||
.collect() | ||
.distinct | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we need |
||
.sortWith(_._2 > _._2) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
} | ||
|
||
/** | ||
* Generate combination of items by computing on FPTree, | ||
* the computation is done on each FPTree partitions. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should document at least the return value. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For the return type, it would be nice to use |
||
*/ | ||
private def generateCombinations( | ||
data: RDD[Array[String]], | ||
minCount: Double, | ||
singleItem: Array[(String, Int)]): Array[(String, Int)] = { | ||
val single = data.context.broadcast(singleItem) | ||
data.flatMap(basket => createFPTree(basket, single)) | ||
.groupByKey() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should it be an |
||
.flatMap(partition => runFPTree(partition, minCount)) | ||
.collect() | ||
} | ||
|
||
/** | ||
* Create FP-Tree partition for the giving basket | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same here: missing docs. |
||
*/ | ||
private def createFPTree( | ||
basket: Array[String], | ||
singleItem: Broadcast[Array[(String, Int)]]): Array[(String, Array[String])] = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You can call |
||
var output = ArrayBuffer[(String, Array[String])]() | ||
var combination = ArrayBuffer[String]() | ||
val single = singleItem.value | ||
var items = ArrayBuffer[(String, Int)]() | ||
|
||
// Filter the basket by single item pattern | ||
val iterator = basket.iterator | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't need an iterator. The code could be simplified to
|
||
while (iterator.hasNext){ | ||
val item = iterator.next | ||
val opt = single.find(_._1.equals(item)) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we need to test membership, should we make |
||
if (opt != None) { | ||
items ++= opt | ||
} | ||
} | ||
|
||
// Sort it and create the item combinations | ||
val sortedItems = items.sortWith(_._1 > _._1).sortWith(_._2 > _._2).toArray | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why sorting twice? The second will overwrite the first. Besides, using |
||
val itemIterator = sortedItems.iterator | ||
while (itemIterator.hasNext) { | ||
combination.clear() | ||
val item = itemIterator.next | ||
val firstNItems = sortedItems.take(sortedItems.indexOf(item)) | ||
if (firstNItems.length > 0) { | ||
val iterator = firstNItems.iterator | ||
while (iterator.hasNext) { | ||
val elem = iterator.next | ||
combination += elem._1 | ||
} | ||
output += ((item._1, combination.toArray)) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't see how this implements the Mapper in Algorithm 4 of the PFP paper. The code here doesn't filter the output. For example, if we have |
||
} | ||
} | ||
output.toArray | ||
} | ||
|
||
/** | ||
* Generate frequent pattern by walking through the FPTree | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missing doc on the input args and the return value. |
||
*/ | ||
private def runFPTree( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we make it a method inside |
||
partition: (String, Iterable[Array[String]]), | ||
minCount: Double): Array[(String, Int)] = { | ||
val key = partition._1 | ||
val value = partition._2 | ||
val output = ArrayBuffer[(String, Int)]() | ||
val map = Map[String, Int]() | ||
|
||
// Walk through the FPTree partition to generate all combinations that satisfy | ||
// the minimal support level. | ||
var k = 1 | ||
while (k > 0) { | ||
map.clear() | ||
val iterator = value.iterator | ||
while (iterator.hasNext) { | ||
val pattern = iterator.next | ||
if (pattern.length >= k) { | ||
val combination = pattern.toList.combinations(k).toList | ||
val itemIterator = combination.iterator | ||
while (itemIterator.hasNext){ | ||
val item = itemIterator.next | ||
val list2key: List[String] = (item :+ key).sortWith(_ > _) | ||
val newKey = list2key.mkString(" ") | ||
if (map.get(newKey) == None) { | ||
map(newKey) = 1 | ||
} else { | ||
map(newKey) = map.apply(newKey) + 1 | ||
} | ||
} | ||
} | ||
} | ||
var eligible: Array[(String, Int)] = null | ||
if (map.size != 0) { | ||
val candidate = map.filter(_._2 >= minCount) | ||
if (candidate.size != 0) { | ||
eligible = candidate.toArray | ||
output ++= eligible | ||
} | ||
} | ||
if ((eligible == null) || (eligible.length == 0)) { | ||
k = 0 | ||
} else { | ||
k = k + 1 | ||
} | ||
} | ||
output.toArray | ||
} | ||
} | ||
|
||
/** | ||
* Top-level methods for calling FPGrowth. | ||
*/ | ||
object FPGrowth{ | ||
|
||
/** | ||
* Generate a FPGrowth Model using the given minimal support level. | ||
* | ||
* @param data input baskets stored as `RDD[Array[String]]` | ||
* @param minSupport minimal support level, for example 0.5 | ||
*/ | ||
def train(data: RDD[Array[String]], minSupport: Double): FPGrowthModel = { | ||
new FPGrowth().setMinSupport(minSupport).run(data) | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,24 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.mllib.fpm | ||
|
||
/** | ||
* A FPGrowth Model for FPGrowth, each element is a frequent pattern with count. | ||
*/ | ||
class FPGrowthModel (val frequentPattern: Array[(String, Int)]) extends Serializable { | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.mllib.fpm | ||
|
||
import org.scalatest.FunSuite | ||
import org.apache.spark.mllib.util.MLlibTestSparkContext | ||
|
||
class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { | ||
|
||
test("test FPGrowth algorithm") | ||
{ | ||
val arr = FPGrowthSuite.createTestData() | ||
|
||
assert(arr.length === 6) | ||
val dataSet = sc.parallelize(arr) | ||
assert(dataSet.count() == 6) | ||
val rdd = dataSet.map(line => line.split(" ")) | ||
assert(rdd.count() == 6) | ||
|
||
val algorithm = new FPGrowth() | ||
algorithm.setMinSupport(0.9) | ||
assert(algorithm.run(rdd).frequentPattern.length == 0) | ||
algorithm.setMinSupport(0.8) | ||
assert(algorithm.run(rdd).frequentPattern.length == 1) | ||
algorithm.setMinSupport(0.7) | ||
assert(algorithm.run(rdd).frequentPattern.length == 1) | ||
algorithm.setMinSupport(0.6) | ||
assert(algorithm.run(rdd).frequentPattern.length == 2) | ||
algorithm.setMinSupport(0.5) | ||
assert(algorithm.run(rdd).frequentPattern.length == 18) | ||
algorithm.setMinSupport(0.4) | ||
assert(algorithm.run(rdd).frequentPattern.length == 18) | ||
algorithm.setMinSupport(0.3) | ||
assert(algorithm.run(rdd).frequentPattern.length == 54) | ||
algorithm.setMinSupport(0.2) | ||
assert(algorithm.run(rdd).frequentPattern.length == 54) | ||
algorithm.setMinSupport(0.1) | ||
assert(algorithm.run(rdd).frequentPattern.length == 625) | ||
} | ||
} | ||
|
||
object FPGrowthSuite | ||
{ | ||
/** | ||
* Create test data set | ||
*/ | ||
def createTestData():Array[String] = | ||
{ | ||
val arr = Array[String]( | ||
"r z h k p", | ||
"z y x w v u t s", | ||
"s x o n r", | ||
"x z y m t s q e", | ||
"z", | ||
"x z y r q t p") | ||
arr | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is no longer needed.