-
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 |
|---|---|---|
|
|
@@ -45,7 +45,11 @@ class PrefixSpan private ( | |
| private var minSupport: Double, | ||
| private var maxPatternLength: Int) extends Logging with Serializable { | ||
|
|
||
| private val maxProjectedDBSizeBeforeLocalProcessing: Long = 10000 | ||
| /** | ||
| * The maximum number of items allowed in a projected database before local processing. If a | ||
| * projected database exceeds this size, another iteration of distributed PrefixSpan is run. | ||
| */ | ||
| private val maxLocalProjDBSize: Long = 10000 | ||
|
|
||
| /** | ||
| * Constructs a default instance with default parameters | ||
|
|
@@ -63,8 +67,7 @@ class PrefixSpan private ( | |
| * Sets the minimal support level (default: `0.1`). | ||
| */ | ||
| def setMinSupport(minSupport: Double): this.type = { | ||
| require(minSupport >= 0 && minSupport <= 1, | ||
| "The minimum support value must be in [0, 1].") | ||
| require(minSupport >= 0 && minSupport <= 1, "The minimum support value must be in [0, 1].") | ||
| this.minSupport = minSupport | ||
| this | ||
| } | ||
|
|
@@ -79,8 +82,7 @@ class PrefixSpan private ( | |
| */ | ||
| def setMaxPatternLength(maxPatternLength: Int): this.type = { | ||
| // TODO: support unbounded pattern length when maxPatternLength = 0 | ||
| require(maxPatternLength >= 1, | ||
| "The maximum pattern length value must be greater than 0.") | ||
| require(maxPatternLength >= 1, "The maximum pattern length value must be greater than 0.") | ||
| this.maxPatternLength = maxPatternLength | ||
| this | ||
| } | ||
|
|
@@ -119,13 +121,13 @@ class PrefixSpan private ( | |
| }.filter(_._2.nonEmpty) | ||
| } | ||
| } | ||
| var (smallPrefixSuffixPairs, largePrefixSuffixPairs) = splitPrefixSuffixPairs(prefixSuffixPairs) | ||
| var (smallPrefixSuffixPairs, largePrefixSuffixPairs) = partitionByProjDBSize(prefixSuffixPairs) | ||
|
|
||
| while (largePrefixSuffixPairs.count() != 0) { | ||
| val (nextPatternAndCounts, nextPrefixSuffixPairs) = | ||
| getPatternCountsAndPrefixSuffixPairs(minCount, largePrefixSuffixPairs) | ||
| largePrefixSuffixPairs.unpersist() | ||
| val (smallerPairsPart, largerPairsPart) = splitPrefixSuffixPairs(nextPrefixSuffixPairs) | ||
| val (smallerPairsPart, largerPairsPart) = partitionByProjDBSize(nextPrefixSuffixPairs) | ||
| largePrefixSuffixPairs = largerPairsPart | ||
| largePrefixSuffixPairs.persist(StorageLevel.MEMORY_AND_DISK) | ||
| smallPrefixSuffixPairs ++= smallerPairsPart | ||
|
|
@@ -136,7 +138,6 @@ class PrefixSpan private ( | |
| val projectedDatabase = smallPrefixSuffixPairs | ||
| // TODO aggregateByKey | ||
| .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 |
||
| .mapValues(_.toArray) | ||
| val nextPatternAndCounts = getPatternsInLocal(minCount, projectedDatabase) | ||
| allPatternAndCounts ++= nextPatternAndCounts | ||
| } | ||
|
|
@@ -145,23 +146,21 @@ class PrefixSpan private ( | |
|
|
||
|
|
||
| /** | ||
| * Split prefix suffix pairs to two parts: | ||
| * Prefixes with projected databases smaller than maxSuffixesBeforeLocalProcessing and | ||
| * Prefixes with projected databases larger than maxSuffixesBeforeLocalProcessing | ||
| * Partitions the prefix-suffix pairs by projected database size. | ||
| * | ||
| * @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 ]) | ||
| * @return prefix-suffix pairs partitioned by whether their projected database size is <= or | ||
| * greater than [[maxLocalProjDBSize]] | ||
| */ | ||
| private def splitPrefixSuffixPairs( | ||
| prefixSuffixPairs: RDD[(List[Int], Array[Int])]): | ||
| (RDD[(List[Int], Array[Int])], RDD[(List[Int], Array[Int])]) = { | ||
| private def partitionByProjDBSize(prefixSuffixPairs: RDD[(List[Int], Array[Int])]) | ||
| : (RDD[(List[Int], Array[Int])], RDD[(List[Int], Array[Int])]) = { | ||
| val prefixToSuffixSize = prefixSuffixPairs | ||
| .aggregateByKey(0)( | ||
| seqOp = { case (count, suffix) => count + suffix.length }, | ||
| combOp = { _ + _ }) | ||
| val smallPrefixes = prefixToSuffixSize | ||
| .filter(_._2 <= maxProjectedDBSizeBeforeLocalProcessing) | ||
| .map(_._1) | ||
| .filter(_._2 <= maxLocalProjDBSize) | ||
| .keys | ||
| .collect() | ||
| .toSet | ||
| val small = prefixSuffixPairs.filter { case (prefix, _) => smallPrefixes.contains(prefix) } | ||
|
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 should collect |
||
|
|
@@ -214,7 +213,7 @@ class PrefixSpan private ( | |
| */ | ||
| private def getPatternsInLocal( | ||
| minCount: Long, | ||
| data: RDD[(List[Int], Array[Array[Int]])]): RDD[(List[Int], Long)] = { | ||
| data: RDD[(List[Int], Iterable[Array[Int]])]): RDD[(List[Int], Long)] = { | ||
| data.flatMap { | ||
| case (prefix, projDB) => | ||
| LocalPrefixSpan.run(minCount, maxPatternLength, prefix.toList.reverse, projDB) | ||
|
|
||
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 leave a TODO to make it configurable with a better default value.
10000may be too small.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.
OK