-
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 19 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 |
|---|---|---|
|
|
@@ -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,6 +45,8 @@ class PrefixSpan private ( | |
| private var minSupport: Double, | ||
| private var maxPatternLength: Int) extends Logging with Serializable { | ||
|
|
||
| private val maxSuffixesBeforeLocalProcessing: Long = 10000 | ||
|
|
||
| /** | ||
| * Constructs a default instance with default parameters | ||
| * {minSupport: `0.1`, maxPatternLength: `10`}. | ||
|
|
@@ -82,20 +86,106 @@ class PrefixSpan private ( | |
| logWarning("Input data is not cached.") | ||
| } | ||
| val minCount = getMinCount(sequences) | ||
| val lengthOnePatternsAndCounts = | ||
| getFreqItemAndCounts(minCount, sequences).collect() | ||
| val prefixAndProjectedDatabase = getPrefixAndProjectedDatabase( | ||
| lengthOnePatternsAndCounts.map(_._1), sequences) | ||
| val groupedProjectedDatabase = prefixAndProjectedDatabase | ||
| .map(x => (x._1.toSeq, x._2)) | ||
| val lengthOnePatternsAndCounts = getFreqItemAndCounts(minCount, sequences) | ||
| val prefixSuffixPairs = getPrefixSuffixPairs( | ||
| lengthOnePatternsAndCounts.map(_._1).collect(), sequences) | ||
| var patternsCount: Long = lengthOnePatternsAndCounts.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 |
||
| var allPatternAndCounts = lengthOnePatternsAndCounts.map(x => (ArrayBuffer(x._1), x._2)) | ||
| var (smallPrefixSuffixPairs, largePrefixSuffixPairs) = | ||
| splitPrefixSuffixPairs(prefixSuffixPairs) | ||
| largePrefixSuffixPairs.persist(StorageLevel.MEMORY_AND_DISK) | ||
|
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. We actually pass over
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, I tested, and the prefixSuffixPairs.persist (7s) is better than largePrefixSuffixPairs.persist (11s) . |
||
| 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 && | ||
| largePrefixSuffixPairs.count() != 0) { | ||
| val (nextPatternAndCounts, nextPrefixSuffixPairs) = | ||
| getPatternCountsAndPrefixSuffixPairs(minCount, largePrefixSuffixPairs) | ||
| 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 |
||
| largePrefixSuffixPairs.unpersist() | ||
| val splitedPrefixSuffixPairs = splitPrefixSuffixPairs(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. nit: split**_t**_ed
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, instead of
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 |
||
| largePrefixSuffixPairs = splitedPrefixSuffixPairs._2 | ||
| largePrefixSuffixPairs.persist(StorageLevel.MEMORY_AND_DISK) | ||
| smallPrefixSuffixPairs = smallPrefixSuffixPairs ++ splitedPrefixSuffixPairs._1 | ||
| 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 (smallPrefixSuffixPairs.count() > 0) { | ||
| val projectedDatabase = smallPrefixSuffixPairs | ||
| .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.map { case (pattern, count) => (pattern.toArray, count) } | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * Split prefix suffix pairs to two parts: | ||
| * suffixes' size less than maxSuffixesBeforeLocalProcessing and | ||
|
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. "Prefixes with projected databases larger than
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 |
||
| * suffixes' size more than maxSuffixesBeforeLocalProcessing | ||
| * @param prefixSuffixPairs prefix (length n) and suffix pairs, | ||
| * @return small size prefix suffix pairs and big size prefix suffix pairs | ||
| * (RDD[prefix, suffix], RDD[prefix, suffix ]) | ||
| */ | ||
| private def splitPrefixSuffixPairs( | ||
| prefixSuffixPairs: RDD[(ArrayBuffer[Int], Array[Int])]): | ||
| (RDD[(ArrayBuffer[Int], Array[Int])], RDD[(ArrayBuffer[Int], Array[Int])]) = { | ||
| val suffixSizeMap = prefixSuffixPairs | ||
| .map(x => (x._1, x._2.length)) | ||
| .reduceByKey(_ + _) | ||
|
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. How about val prefixToSuffixSize = prefixSuffixPairs
.aggregateByKey(0)(
seqOp = { case (count, suffix) => count + suffix.length },
combOp = { _ + _ })
.collect()
.toMap
val smallPrefixes = prefixToSuffixSize
.filter(_._2 <= maxSuffixesBeforeLocalProcessing)
.keys
.toSet
val smallPairs = prefixSuffixPairs.filter { case (prefix, _) => smallPrefixes.contains(prefix) }
(smallPairs, prefixSuffixPairs.subtract(smallPairs))
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. @feynmanliang I compared these two methods. I find your method's running time more than mine. And the result is not correct. I don't know why it was so, please check it, thank you. |
||
| .map(x => (x._2 <= maxSuffixesBeforeLocalProcessing, Set(x._1))) | ||
| .reduceByKey(_ ++ _) | ||
| .collect | ||
| .toMap | ||
| val small = if (suffixSizeMap.contains(true)) { | ||
| prefixSuffixPairs.filter(x => suffixSizeMap(true).contains(x._1)) | ||
| } else { | ||
| prefixSuffixPairs.filter(x => false) | ||
| } | ||
| val large = if (suffixSizeMap.contains(false)) { | ||
| prefixSuffixPairs.filter(x => suffixSizeMap(false).contains(x._1)) | ||
| } else { | ||
| prefixSuffixPairs.filter(x => false) | ||
| } | ||
| (small, large) | ||
| } | ||
|
|
||
| /** | ||
| * Get the pattern and counts, and prefix suffix pairs | ||
| * @param minCount minimum count | ||
| * @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[(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 = prefixAndFrequentItemAndCounts | ||
| .map { case ((prefix, item), count) => (prefix :+ item, count) } | ||
| val prefixToFrequentNextItemsMap = prefixAndFrequentItemAndCounts | ||
| .keys | ||
| .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 | ||
| .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) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -122,37 +212,40 @@ class PrefixSpan private ( | |
| } | ||
|
|
||
| /** | ||
| * Get the frequent prefixes' projected database. | ||
| * Get the frequent prefixes and suffix pairs. | ||
| * @param frequentPrefixes frequent prefixes | ||
| * @param sequences sequences data | ||
| * @return prefixes and projected database | ||
| * @return prefixes and suffix pairs. | ||
| */ | ||
| private def getPrefixAndProjectedDatabase( | ||
| 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) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * calculate the patterns in local. | ||
| * @param minCount the absolute minimum count | ||
| * @param data patterns and projected sequences data data | ||
| * @param data prefixes and projected sequences data data | ||
| * @return patterns | ||
| */ | ||
| private def getPatternsInLocal( | ||
| minCount: Long, | ||
| data: RDD[(Array[Int], Array[Array[Int]])]): RDD[(Array[Int], Long)] = { | ||
| data.flatMap { case (prefix, projDB) => | ||
| LocalPrefixSpan.run(minCount, maxPatternLength, prefix.toList, projDB) | ||
| .map { case (pattern: List[Int], count: Long) => (pattern.toArray.reverse, count) } | ||
| data: RDD[(Array[Int], Array[Array[Int]])]): RDD[(ArrayBuffer[Int], Long)] = { | ||
| data.flatMap { | ||
| case (prefix, projDB) => | ||
| LocalPrefixSpan.run(minCount, maxPatternLength, prefix.toList.reverse, projDB) | ||
| .map { case (pattern: List[Int], count: Long) => | ||
| (pattern.toArray.reverse.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.
This is really
maxProjectedDBSizeBeforeLocalProcessingsince you are counting the number of total items in the projected db not just number of suffixesThere 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