Skip to content
Closed
Changes from 1 commit
Commits
Show all changes
28 commits
Select commit Hold shift + click to select a range
91fd7e6
Add new algorithm PrefixSpan and test file.
zhangjiajin Jul 7, 2015
575995f
Modified the code according to the review comments.
zhangjiajin Jul 8, 2015
951fd42
Delete Prefixspan.scala
zhangjiajin Jul 8, 2015
a2eb14c
Delete PrefixspanSuite.scala
zhangjiajin Jul 8, 2015
89bc368
Fixed a Scala style error.
zhangjiajin Jul 8, 2015
1dd33ad
Modified the code according to the review comments.
zhangjiajin Jul 9, 2015
4c60fb3
Fix some Scala style errors.
zhangjiajin Jul 9, 2015
ba5df34
Fix a Scala style error.
zhangjiajin Jul 9, 2015
574e56c
Add new object LocalPrefixSpan, and do some optimization.
zhangjiajin Jul 10, 2015
ca9c4c8
Modified the code according to the review comments.
zhangjiajin Jul 11, 2015
22b0ef4
Add feature: Collect enough frequent prefixes before projection in Pr…
zhangjiajin Jul 14, 2015
078d410
fix a scala style error.
zhangjiajin Jul 14, 2015
4dd1c8a
initialize file before rebase.
zhangjiajin Jul 15, 2015
a8fde87
Merge branch 'master' of https://github.com/apache/spark
zhangjiajin Jul 15, 2015
6560c69
Add feature: Collect enough frequent prefixes before projection in Pr…
zhangjiajin Jul 15, 2015
baa2885
Modified the code according to the review comments.
zhangjiajin Jul 15, 2015
095aa3a
Modified the code according to the review comments.
zhangjiajin Jul 16, 2015
b07e20c
Merge branch 'master' of https://github.com/apache/spark into Collect…
zhangjiajin Jul 16, 2015
d2250b7
remove minPatternsBeforeLocalProcessing, add maxSuffixesBeforeLocalPr…
zhangjiajin Jul 18, 2015
64271b3
Modified codes according to comments.
zhangjiajin Jul 27, 2015
6e149fa
Fix splitPrefixSuffixPairs
Jul 28, 2015
01c9ae9
Add getters
Jul 28, 2015
cb2a4fc
Inline code for readability
Jul 28, 2015
da0091b
Use lists for prefixes to reuse data
Jul 28, 2015
1235cfc
Use Iterable[Array[_]] over Array[Array[_]] for database
Jul 28, 2015
c2caa5c
Readability improvements and comments
Jul 28, 2015
87fa021
Improve extend prefix readability
Jul 28, 2015
ad23aa9
Merge pull request #1 from feynmanliang/SPARK-8998-collectBeforeLocal
zhangjiajin Jul 29, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Modified the code according to the review comments.
  • Loading branch information
zhangjiajin committed Jul 16, 2015
commit 095aa3a390446205a4d22227b7ed1fbce46f2c93
87 changes: 44 additions & 43 deletions mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -88,66 +90,65 @@ class PrefixSpan private (
val prefixSuffixPairs = getPrefixSuffixPairs(
lengthOnePatternsAndCounts.map(_._1).collect(), sequences)
var patternsCount: Long = lengthOnePatternsAndCounts.count()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove patternsCount since it's no longer necessary and saves one scan of dataset

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK

var allPatternAndCounts = lengthOnePatternsAndCounts.map(x => (Array(x._1), x._2))
var allPatternAndCounts = lengthOnePatternsAndCounts.map(x => (ArrayBuffer(x._1), x._2))
var currentPrefixSuffixPairs = prefixSuffixPairs
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cache() because reused twice in getPatternCountsAndPrefixSuffixPairs

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need patternLength anymore?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK

currentPrefixSuffixPairs = nextPrefixSuffixPairs
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cache() because reused twice in getPatternCountsAndPrefixSuffixPairs

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: ++=

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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()
Copy link
Contributor

Choose a reason for hiding this comment

The 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 patternsCount > minPatternsBeforeShuffle will guarantee that. Better to count the suffixes for each prefix using aggregateByKey before doing local processing.

.map(x => (x._1.toArray, x._2.toArray))
val nextPatternAndCounts = getPatternsInLocal(minCount, projectedDatabase)
allPatternAndCounts = allPatternAndCounts ++ nextPatternAndCounts
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: ++=

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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)
}

/**
Expand Down Expand Up @@ -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)
}
}
Expand All @@ -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) }
}
}