Skip to content
Snippets Groups Projects
Commit add9d1bb authored by Yuhao Yang's avatar Yuhao Yang Committed by Felix Cheung
Browse files

[SPARK-19791][ML] Add doc and example for fpgrowth

## What changes were proposed in this pull request?

Add a new section for fpm
Add Example for FPGrowth in scala and Java

updated: Rewrite transform to be more compact.

## How was this patch tested?

local doc generation.

Author: Yuhao Yang <yuhao.yang@intel.com>

Closes #17130 from hhbyyh/fpmdoc.
parent b28c3bc2
No related branches found
No related tags found
No related merge requests found
...@@ -8,6 +8,8 @@ ...@@ -8,6 +8,8 @@
url: ml-clustering.html url: ml-clustering.html
- text: Collaborative filtering - text: Collaborative filtering
url: ml-collaborative-filtering.html url: ml-collaborative-filtering.html
- text: Frequent Pattern Mining
url: ml-frequent-pattern-mining.html
- text: Model selection and tuning - text: Model selection and tuning
url: ml-tuning.html url: ml-tuning.html
- text: Advanced topics - text: Advanced topics
......
---
layout: global
title: Frequent Pattern Mining
displayTitle: 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.
**Table of Contents**
* This will become a table of contents (this text will be scraped).
{:toc}
## 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 `spark.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 suffixes of transactions,
and hence is more scalable than a single-machine implementation.
We refer users to the papers for more details.
`spark.ml`'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.
* `minConfidence`: minimum confidence for generating Association Rule. Confidence is an indication of how often an
association rule has been found to be true. For example, if in the transactions itemset `X` appears 4 times, `X`
and `Y` co-occur only 2 times, the confidence for the rule `X => Y` is then 2/4 = 0.5. The parameter will not
affect the mining for frequent itemsets, but specify the minimum confidence for generating association rules
from frequent itemsets.
* `numPartitions`: the number of partitions used to distribute the work. By default the param is not set, and
number of partitions of the input dataset is used.
The `FPGrowthModel` provides:
* `freqItemsets`: frequent itemsets in the format of DataFrame("items"[Array], "freq"[Long])
* `associationRules`: association rules generated with confidence above `minConfidence`, in the format of
DataFrame("antecedent"[Array], "consequent"[Array], "confidence"[Double]).
* `transform`: For each transaction in `itemsCol`, the `transform` method will compare its items against the antecedents
of each association rule. If the record contains all the antecedents of a specific association rule, the rule
will be considered as applicable and its consequents will be added to the prediction result. The transform
method will summarize the consequents from all the applicable rules as prediction. The prediction column has
the same data type as `itemsCol` and does not contain existing items in the `itemsCol`.
**Examples**
<div class="codetabs">
<div data-lang="scala" markdown="1">
Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.fpm.FPGrowth) for more details.
{% include_example scala/org/apache/spark/examples/ml/FPGrowthExample.scala %}
</div>
<div data-lang="java" markdown="1">
Refer to the [Java API docs](api/java/org/apache/spark/ml/fpm/FPGrowth.html) for more details.
{% include_example java/org/apache/spark/examples/ml/JavaFPGrowthExample.java %}
</div>
<div data-lang="python" markdown="1">
Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.fpm.FPGrowth) for more details.
{% include_example python/ml/fpgrowth_example.py %}
</div>
<div data-lang="r" markdown="1">
Refer to the [R API docs](api/R/spark.fpGrowth.html) for more details.
{% include_example r/ml/fpm.R %}
</div>
</div>
...@@ -24,7 +24,7 @@ explicitly, which are usually expensive to generate. ...@@ -24,7 +24,7 @@ explicitly, which are usually expensive to generate.
After the second step, the frequent itemsets can be extracted from the FP-tree. After the second step, the frequent itemsets can be extracted from the FP-tree.
In `spark.mllib`, we implemented a parallel version of FP-growth called PFP, In `spark.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). 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, PFP distributes the work of growing FP-trees based on the suffixes of transactions,
and hence more scalable than a single-machine implementation. and hence more scalable than a single-machine implementation.
We refer users to the papers for more details. We refer users to the papers for more details.
......
/*
* 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.ml;
// $example on$
import java.util.Arrays;
import java.util.List;
import org.apache.spark.ml.fpm.FPGrowth;
import org.apache.spark.ml.fpm.FPGrowthModel;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.types.*;
// $example off$
/**
* An example demonstrating FPGrowth.
* Run with
* <pre>
* bin/run-example ml.JavaFPGrowthExample
* </pre>
*/
public class JavaFPGrowthExample {
public static void main(String[] args) {
SparkSession spark = SparkSession
.builder()
.appName("JavaFPGrowthExample")
.getOrCreate();
// $example on$
List<Row> data = Arrays.asList(
RowFactory.create(Arrays.asList("1 2 5".split(" "))),
RowFactory.create(Arrays.asList("1 2 3 5".split(" "))),
RowFactory.create(Arrays.asList("1 2".split(" ")))
);
StructType schema = new StructType(new StructField[]{ new StructField(
"items", new ArrayType(DataTypes.StringType, true), false, Metadata.empty())
});
Dataset<Row> itemsDF = spark.createDataFrame(data, schema);
FPGrowthModel model = new FPGrowth()
.setItemsCol("items")
.setMinSupport(0.5)
.setMinConfidence(0.6)
.fit(itemsDF);
// Display frequent itemsets.
model.freqItemsets().show();
// Display generated association rules.
model.associationRules().show();
// transform examines the input items against all the association rules and summarize the
// consequents as prediction
model.transform(itemsDF).show();
// $example off$
spark.stop();
}
}
#
# 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.
#
# $example on$
from pyspark.ml.fpm import FPGrowth
# $example off$
from pyspark.sql import SparkSession
"""
An example demonstrating FPGrowth.
Run with:
bin/spark-submit examples/src/main/python/ml/fpgrowth_example.py
"""
if __name__ == "__main__":
spark = SparkSession\
.builder\
.appName("FPGrowthExample")\
.getOrCreate()
# $example on$
df = spark.createDataFrame([
(0, [1, 2, 5]),
(1, [1, 2, 3, 5]),
(2, [1, 2])
], ["id", "items"])
fpGrowth = FPGrowth(itemsCol="items", minSupport=0.5, minConfidence=0.6)
model = fpGrowth.fit(df)
# Display frequent itemsets.
model.freqItemsets.show()
# Display generated association rules.
model.associationRules.show()
# transform examines the input items against all the association rules and summarize the
# consequents as prediction
model.transform(df).show()
# $example off$
spark.stop()
/*
* 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.ml
// scalastyle:off println
// $example on$
import org.apache.spark.ml.fpm.FPGrowth
// $example off$
import org.apache.spark.sql.SparkSession
/**
* An example demonstrating FP-Growth.
* Run with
* {{{
* bin/run-example ml.FPGrowthExample
* }}}
*/
object FPGrowthExample {
def main(args: Array[String]): Unit = {
val spark = SparkSession
.builder
.appName(s"${this.getClass.getSimpleName}")
.getOrCreate()
import spark.implicits._
// $example on$
val dataset = spark.createDataset(Seq(
"1 2 5",
"1 2 3 5",
"1 2")
).map(t => t.split(" ")).toDF("items")
val fpgrowth = new FPGrowth().setItemsCol("items").setMinSupport(0.5).setMinConfidence(0.6)
val model = fpgrowth.fit(dataset)
// Display frequent itemsets.
model.freqItemsets.show()
// Display generated association rules.
model.associationRules.show()
// transform examines the input items against all the association rules and summarize the
// consequents as prediction
model.transform(dataset).show()
// $example off$
spark.stop()
}
}
// scalastyle:on println
...@@ -17,7 +17,6 @@ ...@@ -17,7 +17,6 @@
package org.apache.spark.ml.fpm package org.apache.spark.ml.fpm
import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag import scala.reflect.ClassTag
import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.Path
...@@ -54,7 +53,7 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { ...@@ -54,7 +53,7 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol {
/** /**
* Minimal support level of the frequent pattern. [0.0, 1.0]. Any pattern that appears * Minimal support level of the frequent pattern. [0.0, 1.0]. Any pattern that appears
* more than (minSupport * size-of-the-dataset) times will be output * more than (minSupport * size-of-the-dataset) times will be output in the frequent itemsets.
* Default: 0.3 * Default: 0.3
* @group param * @group param
*/ */
...@@ -82,8 +81,8 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { ...@@ -82,8 +81,8 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol {
def getNumPartitions: Int = $(numPartitions) def getNumPartitions: Int = $(numPartitions)
/** /**
* Minimal confidence for generating Association Rule. * Minimal confidence for generating Association Rule. minConfidence will not affect the mining
* Note that minConfidence has no effect during fitting. * for frequent itemsets, but will affect the association rules generation.
* Default: 0.8 * Default: 0.8
* @group param * @group param
*/ */
...@@ -118,7 +117,7 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { ...@@ -118,7 +117,7 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol {
* Recommendation</a>. PFP distributes computation in such a way that each worker executes an * Recommendation</a>. PFP distributes computation in such a way that each worker executes an
* independent group of mining tasks. The FP-Growth algorithm is described in * independent group of mining tasks. The FP-Growth algorithm is described in
* <a href="http://dx.doi.org/10.1145/335191.335372">Han et al., Mining frequent patterns without * <a href="http://dx.doi.org/10.1145/335191.335372">Han et al., Mining frequent patterns without
* candidate generation</a>. Note null values in the feature column are ignored during fit(). * candidate generation</a>. Note null values in the itemsCol column are ignored during fit().
* *
* @see <a href="http://en.wikipedia.org/wiki/Association_rule_learning"> * @see <a href="http://en.wikipedia.org/wiki/Association_rule_learning">
* Association rule learning (Wikipedia)</a> * Association rule learning (Wikipedia)</a>
...@@ -167,7 +166,6 @@ class FPGrowth @Since("2.2.0") ( ...@@ -167,7 +166,6 @@ class FPGrowth @Since("2.2.0") (
} }
val parentModel = mllibFP.run(items) val parentModel = mllibFP.run(items)
val rows = parentModel.freqItemsets.map(f => Row(f.items, f.freq)) val rows = parentModel.freqItemsets.map(f => Row(f.items, f.freq))
val schema = StructType(Seq( val schema = StructType(Seq(
StructField("items", dataset.schema($(itemsCol)).dataType, nullable = false), StructField("items", dataset.schema($(itemsCol)).dataType, nullable = false),
StructField("freq", LongType, nullable = false))) StructField("freq", LongType, nullable = false)))
...@@ -196,7 +194,7 @@ object FPGrowth extends DefaultParamsReadable[FPGrowth] { ...@@ -196,7 +194,7 @@ object FPGrowth extends DefaultParamsReadable[FPGrowth] {
* :: Experimental :: * :: Experimental ::
* Model fitted by FPGrowth. * Model fitted by FPGrowth.
* *
* @param freqItemsets frequent items in the format of DataFrame("items"[Seq], "freq"[Long]) * @param freqItemsets frequent itemsets in the format of DataFrame("items"[Array], "freq"[Long])
*/ */
@Since("2.2.0") @Since("2.2.0")
@Experimental @Experimental
...@@ -244,10 +242,13 @@ class FPGrowthModel private[ml] ( ...@@ -244,10 +242,13 @@ class FPGrowthModel private[ml] (
/** /**
* The transform method first generates the association rules according to the frequent itemsets. * The transform method first generates the association rules according to the frequent itemsets.
* Then for each association rule, it will examine the input items against antecedents and * Then for each transaction in itemsCol, the transform method will compare its items against the
* summarize the consequents as prediction. The prediction column has the same data type as the * antecedents of each association rule. If the record contains all the antecedents of a
* input column(Array[T]) and will not contain existing items in the input column. The null * specific association rule, the rule will be considered as applicable and its consequents
* values in the feature columns are treated as empty sets. * will be added to the prediction result. The transform method will summarize the consequents
* from all the applicable rules as prediction. The prediction column has the same data type as
* the input column(Array[T]) and will not contain existing items in the input column. The null
* values in the itemsCol columns are treated as empty sets.
* WARNING: internally it collects association rules to the driver and uses broadcast for * WARNING: internally it collects association rules to the driver and uses broadcast for
* efficiency. This may bring pressure to driver memory for large set of association rules. * efficiency. This may bring pressure to driver memory for large set of association rules.
*/ */
...@@ -335,13 +336,13 @@ private[fpm] object AssociationRules { ...@@ -335,13 +336,13 @@ private[fpm] object AssociationRules {
/** /**
* Computes the association rules with confidence above minConfidence. * Computes the association rules with confidence above minConfidence.
* @param dataset DataFrame("items", "freq") containing frequent itemset obtained from * @param dataset DataFrame("items"[Array], "freq"[Long]) containing frequent itemsets obtained
* algorithms like [[FPGrowth]]. * from algorithms like [[FPGrowth]].
* @param itemsCol column name for frequent itemsets * @param itemsCol column name for frequent itemsets
* @param freqCol column name for frequent itemsets count * @param freqCol column name for appearance count of the frequent itemsets
* @param minConfidence minimum confidence for the result association rules * @param minConfidence minimum confidence for generating the association rules
* @return a DataFrame("antecedent", "consequent", "confidence") containing the association * @return a DataFrame("antecedent"[Array], "consequent"[Array], "confidence"[Double])
* rules. * containing the association rules.
*/ */
def getAssociationRulesFromFP[T: ClassTag]( def getAssociationRulesFromFP[T: ClassTag](
dataset: Dataset[_], dataset: Dataset[_],
......
...@@ -122,6 +122,8 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul ...@@ -122,6 +122,8 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext with Defaul
.setMinConfidence(0.5678) .setMinConfidence(0.5678)
assert(fpGrowth.getMinSupport === 0.4567) assert(fpGrowth.getMinSupport === 0.4567)
assert(model.getMinConfidence === 0.5678) assert(model.getMinConfidence === 0.5678)
// numPartitions should not have default value.
assert(fpGrowth.isDefined(fpGrowth.numPartitions) === false)
MLTestingUtils.checkCopyAndUids(fpGrowth, model) MLTestingUtils.checkCopyAndUids(fpGrowth, model)
ParamsSuite.checkParams(fpGrowth) ParamsSuite.checkParams(fpGrowth)
ParamsSuite.checkParams(model) ParamsSuite.checkParams(model)
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment