-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-19791] [ML] Add doc and example for fpgrowth #17130
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
Conversation
| val itemset = items.toSet | ||
| brRules.value.flatMap(rule => | ||
| if (items != null && rule._1.forall(item => itemset.contains(item))) { | ||
| brRules.value.flatMap { rule => |
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.
Nit, while we're here -- why change this bit?
Or if simplifying, what about
brRules.value.filter(_._1_forall(itemset.contains)).flatMap(_._2.filter(!itemset.contains(_)))
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.
The change was about a style comment from the orginal PR that I missed. But it's great to see your suggestion. I'll run some test to confirm the performance.
|
Test build #73719 has finished for PR 17130 at commit
|
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.
Just some minor comments. I also ran the examples and output looks good.
| * }}} | ||
| */ | ||
| object FPGrowthExample { | ||
|
|
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.
nit: remove blank line
| object FPGrowthExample { | ||
|
|
||
| def main(args: Array[String]): Unit = { | ||
|
|
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.
nit: remove blank line
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.
Thanks, I'll remove this line.
| import spark.implicits._ | ||
|
|
||
| // $example on$ | ||
| // Loads data. |
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.
I think this comment is pretty obvious, you can probably remove it
| "1 2 5", | ||
| "1 2 3 5", | ||
| "1 2") | ||
| ).map(t => t.split(" ")).toDF("features") |
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.
I think it is better to explicitly declare the data instead of manipulating strings, that way it is very clear what the input data is for the example. On second thought, never mind this comment - it's pretty clear the way it is
| val fpgrowth = new FPGrowth().setMinSupport(0.5).setMinConfidence(0.6) | ||
| val model = fpgrowth.fit(dataset) | ||
|
|
||
| // get frequent itemsets. |
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.
should say "Display frequent itemsets."
| // get frequent itemsets. | ||
| model.freqItemsets.show() | ||
|
|
||
| // get generated association rules. |
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.
same as comment above
| // transform examines the input items against all the association rules and summarize the | ||
| // consequents as prediction | ||
| model.transform(dataset).show() | ||
|
|
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.
nit: remove blank line
| /** | ||
| * Number of partitions (>=1) used by parallel FP-growth. By default the param is not set, and | ||
| * partition number of the input dataset is used. | ||
| * Number of partitions (positive) used by parallel FP-growth. By default the param is not set, |
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.
why change to "positive", I think it was clearer before
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.
I presume to fix a javadoc error because angle brackets are read as opening an HTML tag
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.
Yes. That's the reason. But still I'm getting some java doc error after merging code. Looking into it..
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.
I don't think the error is related to this change.
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.
Let's just use ">=1" but figure out how to escape the characters for javadoc. We'll want to do that long-term.
| "1 2") | ||
| ).map(t => t.split(" ")).toDF("features") | ||
|
|
||
| // Trains a FPGrowth model. |
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.
nit: technically it is just the line below calling fit that trains the model, I would move this comment down or just take it out
|
ping @jkbradley since we're changing the FPGrowth |
|
Test build #73790 has finished for PR 17130 at commit
|
|
Test build #74383 has started for PR 17130 at commit |
|
The updated transform looks good; thanks for pinging! |
|
Test build #3599 has finished for PR 17130 at commit
|
|
Thanks for the review. I'll wait for #17283 to be merged first and resolve the conflict. |
|
Test build #74565 has finished for PR 17130 at commit
|
|
Test build #74636 has finished for PR 17130 at commit
|
|
Refined some comments and minor things. This should be ready for review. Thanks. |
|
|
|
Test build #74994 has finished for PR 17130 at commit
|
|
I'll be happy to help get this merged now that the column renaming is done |
|
Noting here: Please check out the "Issue this PR brought up" here: #17218 It may affect this PR. Thanks! |
|
Test build #75112 has finished for PR 17130 at commit
|
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.
a few remarks regarding the documentation otherwise it looks good 👍
docs/ml-frequent-pattern-mining.md
Outdated
| 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 suffices of transactions, |
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.
suffixes
docs/ml-frequent-pattern-mining.md
Outdated
| 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 suffices of transactions, | ||
| and hence more scalable than a single-machine implementation. |
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.
is more scalable
docs/ml-frequent-pattern-mining.md
Outdated
| * `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. The parameter will not affect the mining | ||
| for frequent itemsets,, but specify the minimum confidence for generating association rules from frequent itemsets. |
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.
It might be good to give an example for confidence as well since one has been given for support
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.
also, there are two commas after itemsets
docs/ml-frequent-pattern-mining.md
Outdated
| * `minConfidence`: minimum confidence for generating Association Rule. 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 | ||
| partition number of the input dataset is used. |
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.
the number of partitions of the input dataset
docs/ml-frequent-pattern-mining.md
Outdated
| * `associationRules`: association rules generated with confidence above `minConfidence`, in the format of | ||
| DataFrame("antecedent"[Array], "consequent"[Array], "confidence"[Double]). | ||
| * `transform`: The transform method examines the input items in `itemsCol` against all the association rules and | ||
| summarize the consequents as prediction. The prediction column has the same data type as the |
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.
I don't think this really explains what transform does or maybe it's just me?
I would have said something like:
The transform method will produce
predictionColcontaining all the consequents of the association rules containing the items initemsColas their antecedents. The prediction column...
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.
Thanks for the suggestion. I do wish to have a better illustration here. But the two containing in your version make it not that straightforward, and actually it should be items in itemsCol contains the antecedents for association rules.
I extend it to a longer version,
For each record 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.
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.
even better 👍
|
Test build #75409 has finished for PR 17130 at commit
|
|
|
||
| fpGrowth = FPGrowth(itemsCol="items", minSupport=0.5, minConfidence=0.6) | ||
| fpGrowthModel = fpGrowth.fit(df) | ||
|
|
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.
Can we associationRules example? After all this is the biggest advantage for the Python user.
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.
definitely. Thanks.
|
Test build #75714 has finished for PR 17130 at commit
|
|
right, how are we on this? let's get this ready soon and merge? |
| 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> |
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.
add R please
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.
Sure. Added reference to R example. Manually checked on generated doc.
|
Test build #75947 has finished for PR 17130 at commit
|
docs/ml-frequent-pattern-mining.md
Outdated
| * `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 |
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.
itemsCol?
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.
Please refer to https://issues.apache.org/jira/browse/SPARK-19899
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.
I mean style it as code with backtick
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.
sure.
| /** | ||
| * Minimal confidence for generating Association Rule. | ||
| * Note that minConfidence has no effect during fitting. | ||
| * Minimal confidence for generating Association Rule. MinConfidence will not affect the mining |
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.
lower case minConfidence?
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.
ping
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.
got it.
| Seq.empty | ||
| }).distinct | ||
| brRules.value.filter(_._1.forall(itemset.contains)) | ||
| .flatMap(_._2.filter(!itemset.contains(_))).distinct |
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.
are we aware of code changes here
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.
so we don't need to handle null item?
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.
Hi @felixcheung , can you be more specific about the null item that concerns you? Thanks.
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.
right, 2 things - first just calling out while the PR says doc changes there is this one code change here.
second, before this code was checking items != null do we need not consider that now?
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.
items != null is already checked at two lines above.
Please refer to the comments in the PR for the history of the code change. I can update title to include the code change.
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.
let's update the PR/JIRA if code change is required for the doc change.
otherwise, let's leave code change as a separate PR?
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.
I guess that's the right way. I will revert the code change.
|
@felixcheung, reverted the code change of |
|
LGTM, thanks for adding this! @felixcheung OK with merging? |
|
Test build #76287 has finished for PR 17130 at commit
|
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.
LGTM
|
merged to master/2.2 |
## 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 <[email protected]> Closes #17130 from hhbyyh/fpmdoc. (cherry picked from commit add9d1b) Signed-off-by: Felix Cheung <[email protected]>
## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-20670 As suggested by Sean Owen in apache#17130, the transform code in FPGrowthModel can be simplified. As I tested on some public dataset http://fimi.ua.ac.be/data/, the performance of the new transform code is even or better than the old implementation. ## How was this patch tested? Existing unit test. Author: Yuhao Yang <[email protected]> Closes apache#17912 from hhbyyh/fpgrowthTransform.
## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-20670 As suggested by Sean Owen in apache#17130, the transform code in FPGrowthModel can be simplified. As I tested on some public dataset http://fimi.ua.ac.be/data/, the performance of the new transform code is even or better than the old implementation. ## How was this patch tested? Existing unit test. Author: Yuhao Yang <[email protected]> Closes apache#17912 from hhbyyh/fpgrowthTransform.
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.