aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--docs/mllib-frequent-pattern-mining.md100
-rw-r--r--docs/mllib-guide.md2
-rw-r--r--examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java63
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala51
4 files changed, 216 insertions, 0 deletions
diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md
new file mode 100644
index 0000000000..0ff9738768
--- /dev/null
+++ b/docs/mllib-frequent-pattern-mining.md
@@ -0,0 +1,100 @@
+---
+layout: global
+title: Frequent Pattern Mining - MLlib
+displayTitle: <a href="mllib-guide.html">MLlib</a> - Frequent Pattern Mining
+---
+
+Mining frequent items, itemsets, subsequences, or other substructures is usually among the
+first steps to analyze a large-scale dataset, which has been an active research topic in
+data mining for years.
+We refer users to Wikipedia's [association rule learning](http://en.wikipedia.org/wiki/Association_rule_learning)
+for more information.
+MLlib provides a parallel implementation of FP-growth,
+a popular algorithm to mining frequent itemsets.
+
+## FP-growth
+
+The FP-growth algorithm is described in the paper
+[Han et al., Mining frequent patterns without candidate generation](http://dx.doi.org/10.1145/335191.335372),
+where "FP" stands for frequent pattern.
+Given a dataset of transactions, the first step of FP-growth is to calculate item frequencies and identify frequent items.
+Different from [Apriori-like](http://en.wikipedia.org/wiki/Apriori_algorithm) algorithms designed for the same purpose,
+the second step of FP-growth uses a suffix tree (FP-tree) structure to encode transactions without generating candidate sets
+explicitly, which are usually expensive to generate.
+After the second step, the frequent itemsets can be extracted from the FP-tree.
+In MLlib, we implemented a parallel version of FP-growth called PFP,
+as described in [Li et al., PFP: Parallel FP-growth for query recommendation](http://dx.doi.org/10.1145/1454008.1454027).
+PFP distributes the work of growing FP-trees based on the suffices of transactions,
+and hence more scalable than a single-machine implementation.
+We refer users to the papers for more details.
+
+MLlib's FP-growth implementation takes the following (hyper-)parameters:
+
+* `minSupport`: the minimum support for an itemset to be identified as frequent.
+ For example, if an item appears 3 out of 5 transactions, it has a support of 3/5=0.6.
+* `numPartitions`: the number of partitions used to distribute the work.
+
+**Examples**
+
+<div class="codetabs">
+<div data-lang="scala" markdown="1">
+
+[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the
+FP-growth algorithm.
+It take a `JavaRDD` of transactions, where each transaction is an `Iterable` of items of a generic type.
+Calling `FPGrowth.run` with transactions returns an
+[`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html)
+that stores the frequent itemsets with their frequencies.
+
+{% highlight scala %}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel}
+
+val transactions: RDD[Array[String]] = ...
+
+val fpg = new FPGrowth()
+ .setMinSupport(0.2)
+ .setNumPartitions(10)
+val model = fpg.run(transactions)
+
+model.freqItemsets.collect().foreach { case (itemset, freq) =>
+ println(itemset.mkString("[", ",", "]") + ", " + freq)
+}
+{% endhighlight %}
+
+</div>
+
+<div data-lang="java" markdown="1">
+
+[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the
+FP-growth algorithm.
+It take an `RDD` of transactions, where each transaction is an `Array` of items of a generic type.
+Calling `FPGrowth.run` with transactions returns an
+[`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html)
+that stores the frequent itemsets with their frequencies.
+
+{% highlight java %}
+import java.util.Arrays;
+import java.util.List;
+
+import scala.Tuple2;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.mllib.fpm.FPGrowth;
+import org.apache.spark.mllib.fpm.FPGrowthModel;
+
+JavaRDD<List<String>> transactions = ...
+
+FPGrowth fpg = new FPGrowth()
+ .setMinSupport(0.2)
+ .setNumPartitions(10);
+
+FPGrowthModel<String> model = fpg.run(transactions);
+
+for (Tuple2<Object, Long> s: model.javaFreqItemsets().collect()) {
+ System.out.println("(" + Arrays.toString((Object[]) s._1()) + "): " + s._2());
+}
+{% endhighlight %}
+
+</div>
+</div>
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index fbe809b347..0ca51f92d7 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -34,6 +34,8 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv
* singular value decomposition (SVD)
* principal component analysis (PCA)
* [Feature extraction and transformation](mllib-feature-extraction.html)
+* [Frequent pattern mining](mllib-frequent-pattern-mining.html)
+ * FP-growth
* [Optimization (developer)](mllib-optimization.html)
* stochastic gradient descent
* limited-memory BFGS (L-BFGS)
diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java
new file mode 100644
index 0000000000..0db572d760
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaFPGrowthExample.java
@@ -0,0 +1,63 @@
+/*
+ * 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.examples.mllib;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import scala.Tuple2;
+
+import com.google.common.collect.Lists;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.fpm.FPGrowth;
+import org.apache.spark.mllib.fpm.FPGrowthModel;
+
+/**
+ * Java example for mining frequent itemsets using FP-growth.
+ */
+public class JavaFPGrowthExample {
+
+ public static void main(String[] args) {
+ SparkConf sparkConf = new SparkConf().setAppName("JavaFPGrowthExample");
+ JavaSparkContext sc = new JavaSparkContext(sparkConf);
+
+
+ // TODO: Read a user-specified input file.
+ @SuppressWarnings("unchecked")
+ JavaRDD<ArrayList<String>> transactions = sc.parallelize(Lists.newArrayList(
+ Lists.newArrayList("r z h k p".split(" ")),
+ Lists.newArrayList("z y x w v u t s".split(" ")),
+ Lists.newArrayList("s x o n r".split(" ")),
+ Lists.newArrayList("x z y m t s q e".split(" ")),
+ Lists.newArrayList("z".split(" ")),
+ Lists.newArrayList("x z y r q t p".split(" "))), 2);
+
+ FPGrowth fpg = new FPGrowth()
+ .setMinSupport(0.3);
+ FPGrowthModel<String> model = fpg.run(transactions);
+
+ for (Tuple2<Object, Long> s: model.javaFreqItemsets().collect()) {
+ System.out.println(Arrays.toString((Object[]) s._1()) + ", " + s._2());
+ }
+
+ sc.stop();
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
new file mode 100644
index 0000000000..ae66107d70
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.examples.mllib
+
+import org.apache.spark.mllib.fpm.FPGrowth
+import org.apache.spark.{SparkContext, SparkConf}
+
+/**
+ * Example for mining frequent itemsets using FP-growth.
+ */
+object FPGrowthExample {
+
+ def main(args: Array[String]) {
+ val conf = new SparkConf().setAppName("FPGrowthExample")
+ val sc = new SparkContext(conf)
+
+ // TODO: Read a user-specified input file.
+ val transactions = sc.parallelize(Seq(
+ "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").map(_.split(" ")), numSlices = 2)
+
+ val fpg = new FPGrowth()
+ .setMinSupport(0.3)
+ val model = fpg.run(transactions)
+
+ model.freqItemsets.collect().foreach { case (itemset, freq) =>
+ println(itemset.mkString("[", ",", "]") + ", " + freq)
+ }
+
+ sc.stop()
+ }
+}