Skip to content
Closed
Show file tree
Hide file tree
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
Use Iterable[Array[_]] over Array[Array[_]] for database
  • Loading branch information
Feynman Liang committed Jul 28, 2015
commit 1235cfcc9367b546bcf564972a33b769f62da520
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ private[fpm] object LocalPrefixSpan extends Logging with Serializable {
minCount: Long,
maxPatternLength: Int,
prefixes: List[Int],
database: Array[Array[Int]]): Iterator[(List[Int], Long)] = {
database: Iterable[Array[Int]]): Iterator[(List[Int], Long)] = {
if (prefixes.length == maxPatternLength || database.isEmpty) return Iterator.empty
val frequentItemAndCounts = getFreqItemAndCounts(minCount, database)
val filteredDatabase = database.map(x => x.filter(frequentItemAndCounts.contains))
Expand All @@ -67,7 +67,7 @@ private[fpm] object LocalPrefixSpan extends Logging with Serializable {
}
}

def project(database: Array[Array[Int]], prefix: Int): Array[Array[Int]] = {
def project(database: Iterable[Array[Int]], prefix: Int): Iterable[Array[Int]] = {
database
.map(getSuffix(prefix, _))
.filter(_.nonEmpty)
Expand All @@ -81,7 +81,7 @@ private[fpm] object LocalPrefixSpan extends Logging with Serializable {
*/
private def getFreqItemAndCounts(
minCount: Long,
database: Array[Array[Int]]): mutable.Map[Int, Long] = {
database: Iterable[Array[Int]]): mutable.Map[Int, Long] = {
// TODO: use PrimitiveKeyOpenHashMap
val counts = mutable.Map[Int, Long]().withDefaultValue(0L)
database.foreach { sequence =>
Expand Down
37 changes: 18 additions & 19 deletions mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Copy link
Contributor

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. 10000 may be too small.

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


/**
* Constructs a default instance with default parameters
Expand All @@ -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
}
Expand All @@ -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
}
Expand Down Expand Up @@ -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
Expand All @@ -136,7 +138,6 @@ class PrefixSpan private (
val projectedDatabase = smallPrefixSuffixPairs
// TODO aggregateByKey
.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.

.mapValues(_.toArray)
val nextPatternAndCounts = getPatternsInLocal(minCount, projectedDatabase)
allPatternAndCounts ++= nextPatternAndCounts
}
Expand All @@ -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) }
Copy link
Contributor

Choose a reason for hiding this comment

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

We should collect small to local (assuming it is indeed small). Otherwise, the final RDD would have really complex lineage.

Expand Down Expand Up @@ -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)
Expand Down