Github user mengxr commented on a diff in the pull request:
https://github.com/apache/spark/pull/7412#discussion_r35735883
--- Diff: mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala
---
@@ -81,78 +99,145 @@ class PrefixSpan private (
if (sequences.getStorageLevel == StorageLevel.NONE) {
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))
- .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
+
+ // Convert min support to a min number of transactions for this dataset
+ val minCount = if (minSupport == 0) 0L else
math.ceil(sequences.count() * minSupport).toLong
+
+ // (Frequent items -> number of occurrences, all items here satisfy
the `minSupport` threshold
+ val freqItemCounts = sequences
+ .flatMap(seq => seq.distinct.map(item => (item, 1L)))
+ .reduceByKey(_ + _)
+ .filter(_._2 >= minCount)
+
+ // Pairs of (length 1 prefix, suffix consisting of frequent items)
+ val itemSuffixPairs = {
+ val freqItems = freqItemCounts.keys.collect().toSet
+ sequences.flatMap { seq =>
+ val filteredSeq = seq.filter(freqItems.contains(_))
+ freqItems.flatMap { item =>
+ val candidateSuffix = LocalPrefixSpan.getSuffix(item,
filteredSeq)
+ candidateSuffix match {
+ case suffix if !suffix.isEmpty => Some((List(item), suffix))
+ case _ => None
+ }
+ }
+ }
+ }
+
+ // Accumulator for the computed results to be returned, initialized to
the frequent items (i.e.
+ // frequent length-one prefixes)
+ var resultsAccumulator = freqItemCounts.map(x => (List(x._1), x._2))
+
+ // Remaining work to be locally and distributively processed
respectfully
+ var (pairsForLocal, pairsForDistributed) =
partitionByProjDBSize(itemSuffixPairs)
+
+ // Continue processing until no pairs for distributed processing
remain (i.e. all prefixes have
+ // projected database sizes <= `maxLocalProjDBSize`)
+ while (pairsForDistributed.count() != 0) {
+ val (nextPatternAndCounts, nextPrefixSuffixPairs) =
+ extendPrefixes(minCount, pairsForDistributed)
+ pairsForDistributed.unpersist()
+ val (smallerPairsPart, largerPairsPart) =
partitionByProjDBSize(nextPrefixSuffixPairs)
+ pairsForDistributed = largerPairsPart
+ pairsForDistributed.persist(StorageLevel.MEMORY_AND_DISK)
+ pairsForLocal ++= smallerPairsPart
+ resultsAccumulator ++= nextPatternAndCounts
+ }
+
+ // Process the small projected databases locally
+ resultsAccumulator ++= getPatternsInLocal(minCount,
pairsForLocal.groupByKey())
+
+ resultsAccumulator.map { case (pattern, count) => (pattern.toArray,
count) }
}
+
/**
- * Get the minimum count (sequences count * minSupport).
- * @param sequences input data set, contains a set of sequences,
- * @return minimum count,
+ * Partitions the prefix-suffix pairs by projected database size.
+ * @param prefixSuffixPairs prefix (length n) and suffix pairs,
+ * @return prefix-suffix pairs partitioned by whether their projected
database size is <= or
+ * greater than [[maxLocalProjDBSize]]
*/
- private def getMinCount(sequences: RDD[Array[Int]]): Long = {
- if (minSupport == 0) 0L else math.ceil(sequences.count() *
minSupport).toLong
+ 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 <= maxLocalProjDBSize)
+ .keys
+ .collect()
+ .toSet
+ val small = prefixSuffixPairs.filter { case (prefix, _) =>
smallPrefixes.contains(prefix) }
--- End diff --
We should collect `small` to local (assuming it is indeed small).
Otherwise, the final RDD would have really complex lineage.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]