-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-8998][MLlib] Collect enough frequent prefixes before local processing in PrefixSpan (new) #7412
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-8998][MLlib] Collect enough frequent prefixes before local processing in PrefixSpan (new) #7412
Changes from 1 commit
91fd7e6
575995f
951fd42
a2eb14c
89bc368
1dd33ad
4c60fb3
ba5df34
574e56c
ca9c4c8
22b0ef4
078d410
4dd1c8a
a8fde87
6560c69
baa2885
095aa3a
b07e20c
d2250b7
64271b3
6e149fa
01c9ae9
cb2a4fc
da0091b
1235cfc
c2caa5c
87fa021
ad23aa9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,8 @@ | |
|
|
||
| package org.apache.spark.mllib.fpm | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.annotation.Experimental | ||
| import org.apache.spark.rdd.RDD | ||
|
|
@@ -43,7 +45,7 @@ class PrefixSpan private ( | |
| private var minSupport: Double, | ||
| private var maxPatternLength: Int) extends Logging with Serializable { | ||
|
|
||
| private val minPatternsBeforeShuffle: Int = 20 | ||
| private val minPatternsBeforeLocalProcessing: Int = 20 | ||
|
|
||
| /** | ||
| * Constructs a default instance with default parameters | ||
|
|
@@ -88,66 +90,65 @@ class PrefixSpan private ( | |
| val prefixSuffixPairs = getPrefixSuffixPairs( | ||
| lengthOnePatternsAndCounts.map(_._1).collect(), sequences) | ||
| var patternsCount: Long = lengthOnePatternsAndCounts.count() | ||
| var allPatternAndCounts = lengthOnePatternsAndCounts.map(x => (Array(x._1), x._2)) | ||
| var allPatternAndCounts = lengthOnePatternsAndCounts.map(x => (ArrayBuffer(x._1), x._2)) | ||
| var currentPrefixSuffixPairs = prefixSuffixPairs | ||
|
Contributor
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.
Contributor
Author
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. OK. Because the pairs maybe very big, so, I use persist(StorageLevel.MEMORY_AND_DISK). |
||
| while (patternsCount <= minPatternsBeforeShuffle && currentPrefixSuffixPairs.count() != 0) { | ||
| var patternLength: Int = 1 | ||
|
Contributor
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
Contributor
Author
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. OK |
||
| while (patternLength < maxPatternLength && | ||
| patternsCount <= minPatternsBeforeLocalProcessing && | ||
| currentPrefixSuffixPairs.count() != 0) { | ||
| val (nextPatternAndCounts, nextPrefixSuffixPairs) = | ||
| getPatternCountsAndPrefixSuffixPairs(minCount, currentPrefixSuffixPairs) | ||
| patternsCount = nextPatternAndCounts.count().toInt | ||
| patternsCount = nextPatternAndCounts.count() | ||
|
Contributor
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. Remove
Contributor
Author
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. OK |
||
| currentPrefixSuffixPairs = nextPrefixSuffixPairs | ||
|
Contributor
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.
Contributor
Author
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. OK. Because the pairs maybe very big, so, I use persist(StorageLevel.MEMORY_AND_DISK). |
||
| allPatternAndCounts = allPatternAndCounts ++ nextPatternAndCounts | ||
|
Contributor
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. nit:
Contributor
Author
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. OK |
||
| patternLength = patternLength + 1 | ||
| } | ||
| if (patternsCount > 0) { | ||
| if (patternLength < maxPatternLength && patternsCount > 0) { | ||
| val projectedDatabase = currentPrefixSuffixPairs | ||
| .map(x => (x._1.toSeq, x._2)) | ||
| .groupByKey() | ||
|
Contributor
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. This assumes that all the values (suffixes) associated to a key (prefix) will fit on an executor, but I don't think that |
||
| .map(x => (x._1.toArray, x._2.toArray)) | ||
| val nextPatternAndCounts = getPatternsInLocal(minCount, projectedDatabase) | ||
| allPatternAndCounts = allPatternAndCounts ++ nextPatternAndCounts | ||
|
Contributor
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. nit:
Contributor
Author
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. OK |
||
| } | ||
| allPatternAndCounts | ||
| allPatternAndCounts.map { case (pattern, count) => (pattern.toArray, count) } | ||
| } | ||
|
|
||
| /** | ||
| * Get the pattern and counts, and prefix suffix pairs | ||
| * @param minCount minimum count | ||
| * @param prefixSuffixPairs prefix and suffix pairs, | ||
| * @return pattern and counts, and prefix suffix pairs | ||
| * (Array[pattern, count], RDD[prefix, suffix ]) | ||
| * @param prefixSuffixPairs prefix (length n) and suffix pairs, | ||
| * @return pattern (length n+1) and counts, and prefix (length n+1) and suffix pairs | ||
| * (RDD[pattern, count], RDD[prefix, suffix ]) | ||
| */ | ||
| private def getPatternCountsAndPrefixSuffixPairs( | ||
| minCount: Long, | ||
| prefixSuffixPairs: RDD[(Array[Int], Array[Int])]): | ||
| (RDD[(Array[Int], Long)], RDD[(Array[Int], Array[Int])]) = { | ||
| val prefixAndFreqentItemAndCounts = prefixSuffixPairs | ||
| .flatMap { case (prefix, suffix) => | ||
| suffix.distinct.map(y => ((prefix.toSeq, y), 1L)) | ||
| }.reduceByKey(_ + _) | ||
| prefixSuffixPairs: RDD[(ArrayBuffer[Int], Array[Int])]): | ||
| (RDD[(ArrayBuffer[Int], Long)], RDD[(ArrayBuffer[Int], Array[Int])]) = { | ||
| val prefixAndFrequentItemAndCounts = prefixSuffixPairs | ||
| .flatMap { case (prefix, suffix) => suffix.distinct.map(y => ((prefix, y), 1L)) } | ||
| .reduceByKey(_ + _) | ||
| .filter(_._2 >= minCount) | ||
| val patternAndCounts = prefixAndFreqentItemAndCounts | ||
| .map{ case ((prefix, item), count) => (prefix.toArray :+ item, count) } | ||
| val prefixlength = prefixSuffixPairs.first()._1.length | ||
| if (prefixlength + 1 >= maxPatternLength) { | ||
| (patternAndCounts, prefixSuffixPairs.filter(x => false)) | ||
| } else { | ||
| val frequentItemsMap = prefixAndFreqentItemAndCounts | ||
| .keys | ||
| .groupByKey() | ||
| .mapValues(_.toSet) | ||
| .collect | ||
| .toMap | ||
| val nextPrefixSuffixPairs = prefixSuffixPairs | ||
| .filter(x => frequentItemsMap.contains(x._1)) | ||
| .flatMap { case (prefix, suffix) => | ||
| val frequentItemSet = frequentItemsMap(prefix) | ||
| val filteredSuffix = suffix.filter(frequentItemSet.contains(_)) | ||
| val nextSuffixes = frequentItemSet.map{ item => | ||
| (item, LocalPrefixSpan.getSuffix(item, filteredSuffix)) | ||
| }.filter(_._2.nonEmpty) | ||
| nextSuffixes.map { case (item, suffix) => (prefix :+ item, suffix) } | ||
| val patternAndCounts = prefixAndFrequentItemAndCounts | ||
| .map { case ((prefix, item), count) => (prefix :+ item, count) } | ||
| val prefixToFrequentNextItemsMap = prefixAndFrequentItemAndCounts | ||
| .keys | ||
| .groupByKey() | ||
| .mapValues(_.toSet) | ||
| .collect() | ||
| .toMap | ||
| val nextPrefixSuffixPairs = prefixSuffixPairs | ||
| .filter(x => prefixToFrequentNextItemsMap.contains(x._1)) | ||
| .flatMap { case (prefix, suffix) => | ||
| val frequentNextItems = prefixToFrequentNextItemsMap(prefix) | ||
| val filteredSuffix = suffix.filter(frequentNextItems.contains(_)) | ||
| frequentNextItems.flatMap { item => | ||
| val suffix = LocalPrefixSpan.getSuffix(item, filteredSuffix) | ||
| if (suffix.isEmpty) None | ||
| else Some(prefix :+ item, suffix) | ||
| } | ||
| (patternAndCounts, nextPrefixSuffixPairs) | ||
| } | ||
| (patternAndCounts, nextPrefixSuffixPairs) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -181,14 +182,14 @@ class PrefixSpan private ( | |
| */ | ||
| private def getPrefixSuffixPairs( | ||
| frequentPrefixes: Array[Int], | ||
| sequences: RDD[Array[Int]]): RDD[(Array[Int], Array[Int])] = { | ||
| sequences: RDD[Array[Int]]): RDD[(ArrayBuffer[Int], Array[Int])] = { | ||
| val filteredSequences = sequences.map { p => | ||
| p.filter (frequentPrefixes.contains(_) ) | ||
| } | ||
| filteredSequences.flatMap { x => | ||
| frequentPrefixes.map { y => | ||
| val sub = LocalPrefixSpan.getSuffix(y, x) | ||
| (Array(y), sub) | ||
| (ArrayBuffer(y), sub) | ||
| }.filter(_._2.nonEmpty) | ||
| } | ||
| } | ||
|
|
@@ -201,9 +202,9 @@ class PrefixSpan private ( | |
| */ | ||
| private def getPatternsInLocal( | ||
| minCount: Long, | ||
| data: RDD[(Array[Int], Array[Array[Int]])]): RDD[(Array[Int], Long)] = { | ||
| data.flatMap { x => | ||
| LocalPrefixSpan.run(minCount, maxPatternLength, x._1, x._2) | ||
| } | ||
| data: RDD[(Array[Int], Array[Array[Int]])]): RDD[(ArrayBuffer[Int], Long)] = { | ||
| data | ||
| .flatMap { x => LocalPrefixSpan.run(minCount, maxPatternLength, x._1, x._2) } | ||
| .map { case (pattern, count) => (pattern.to[ArrayBuffer], count) } | ||
| } | ||
| } | ||
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.
Remove
patternsCountsince it's no longer necessary and saves one scan of datasetThere 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.
OK