-
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 15 commits
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
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -43,6 +43,8 @@ class PrefixSpan private ( | |
| private var minSupport: Double, | ||
| private var maxPatternLength: Int) extends Logging with Serializable { | ||
|
|
||
| private val minPatternsBeforeShuffle: Int = 20 | ||
|
|
||
| /** | ||
| * Constructs a default instance with default parameters | ||
| * {minSupport: `0.1`, maxPatternLength: `10`}. | ||
|
|
@@ -86,16 +88,69 @@ class PrefixSpan private ( | |
| getFreqItemAndCounts(minCount, sequences).collect() | ||
| val prefixAndProjectedDatabase = getPrefixAndProjectedDatabase( | ||
|
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. Every reference of "projected database" before L105 should be renamed to "prefixSuffixPairs" since the
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. |
||
| lengthOnePatternsAndCounts.map(_._1), sequences) | ||
| val groupedProjectedDatabase = prefixAndProjectedDatabase | ||
| .map(x => (x._1.toSeq, x._2)) | ||
| .groupByKey() | ||
| .map(x => (x._1.toArray, x._2.toArray)) | ||
| val nextPatterns = getPatternsInLocal(minCount, groupedProjectedDatabase) | ||
| val lengthOnePatternsAndCountsRdd = | ||
| sequences.sparkContext.parallelize( | ||
| lengthOnePatternsAndCounts.map(x => (Array(x._1), x._2))) | ||
| val allPatterns = lengthOnePatternsAndCountsRdd ++ nextPatterns | ||
| allPatterns | ||
|
|
||
| var patternsCount = lengthOnePatternsAndCounts.length | ||
| var allPatternAndCounts = sequences.sparkContext.parallelize( | ||
|
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. No need to
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 |
||
| lengthOnePatternsAndCounts.map(x => (Array(x._1), x._2))) | ||
| var currentProjectedDatabase = prefixAndProjectedDatabase | ||
| while (patternsCount <= minPatternsBeforeShuffle && | ||
| currentProjectedDatabase.count() != 0) { | ||
|
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. No need for linebreak
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 |
||
| val (nextPatternAndCounts, nextProjectedDatabase) = | ||
| getPatternCountsAndProjectedDatabase(minCount, currentProjectedDatabase) | ||
| patternsCount = nextPatternAndCounts.count().toInt | ||
|
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. I think just declaring
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
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. No need for
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 |
||
| currentProjectedDatabase = nextProjectedDatabase | ||
| 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 |
||
| } | ||
| if (patternsCount > 0) { | ||
| val groupedProjectedDatabase = currentProjectedDatabase | ||
|
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 is the
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 |
||
| .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, groupedProjectedDatabase) | ||
| 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 | ||
| } | ||
|
|
||
| /** | ||
| * Get the pattern and counts, and projected database | ||
|
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. "projected database" -> "suffix", ditto for all other occurrences in this method (projected database is all suffixes for a prefix and is an Array[Array[Int]]; the groupBy hasn't happened yet in this method)
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 |
||
| * @param minCount minimum count | ||
| * @param prefixAndProjectedDatabase prefix and projected database, | ||
| * @return pattern and counts, and projected database | ||
| * (Array[pattern, count], RDD[prefix, projected database ]) | ||
| */ | ||
| private def getPatternCountsAndProjectedDatabase( | ||
| minCount: Long, | ||
| prefixAndProjectedDatabase: RDD[(Array[Int], Array[Int])]): | ||
| (RDD[(Array[Int], Long)], RDD[(Array[Int], Array[Int])]) = { | ||
| val prefixAndFreqentItemAndCounts = prefixAndProjectedDatabase.flatMap{ x => | ||
| x._2.distinct.map(y => ((x._1.toSeq, y), 1L)) | ||
| }.reduceByKey(_ + _) | ||
| .filter(_._2 >= minCount) | ||
| val patternAndCounts = prefixAndFreqentItemAndCounts | ||
| .map(x => (x._1._1.toArray ++ Array(x._1._2), x._2)) | ||
|
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 |
||
| val prefixlength = prefixAndProjectedDatabase.take(1)(0)._1.length | ||
|
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. It is not easy that we move prefix length check into the loop on L96.
if we move the check into the loop, we must split getPatternCountsAndPrefixSuffixPairs() to two method: getPatternCounts() and getPrefixSuffixPairs(),
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. Continuing discussion on the latest diff. |
||
| if (prefixlength + 1 >= maxPatternLength) { | ||
|
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
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. Actually doesn't prefixlength only grow by 1 each iteration of L93? You could just use a counter there
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 |
||
| (patternAndCounts, prefixAndProjectedDatabase.filter(x => false)) | ||
| } else { | ||
| val frequentItemsMap = prefixAndFreqentItemAndCounts | ||
|
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 |
||
| .keys.map(x => (x._1, x._2)) | ||
|
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. No need for the
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 |
||
| .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.
|
||
| .mapValues(_.toSet) | ||
| .collect | ||
| .toMap | ||
| val nextPrefixAndProjectedDatabase = prefixAndProjectedDatabase | ||
| .filter(x => frequentItemsMap.contains(x._1)) | ||
| .flatMap { x => | ||
|
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 |
||
| val frequentItemSet = frequentItemsMap(x._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.
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 |
||
| val filteredSequence = x._2.filter(frequentItemSet.contains(_)) | ||
|
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 |
||
| val subProjectedDabase = frequentItemSet.map{ y => | ||
|
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. Space before
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 |
||
| (y, LocalPrefixSpan.getSuffix(y, filteredSequence)) | ||
| }.filter(_._2.nonEmpty) | ||
| subProjectedDabase.map(y => (x._1 ++ Array(y._1), y._2)) | ||
| } | ||
| (patternAndCounts, nextPrefixAndProjectedDatabase) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
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.
minPatternsBeforeLocalProcessingThere 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