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
Next Next commit
Add new algorithm PrefixSpan and test file.
  • Loading branch information
zhangjiajin committed Jul 7, 2015
commit 91fd7e66d0c363e68bc9ebe2bf3e03c26ef348d2
183 changes: 183 additions & 0 deletions mllib/src/main/scala/org/apache/spark/mllib/fpm/Prefixspan.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,183 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.mllib.fpm

import org.apache.spark.rdd.RDD

/**
*
* A parallel PrefixSpan algorithm to mine sequential pattern.
* The PrefixSpan algorithm is described in
* [[http://web.engr.illinois.edu/~hanj/pdf/span01.pdf]].
*
* @param sequences original sequences data
* @param minSupport the minimal support level of the sequential pattern, any pattern appears
* more than minSupport times will be output
* @param maxPatternLength the maximal length of the sequential pattern, any pattern appears
* less than maxPatternLength will be output
*
* @see [[https://en.wikipedia.org/wiki/Sequential_Pattern_Mining Sequential Pattern Mining
* (Wikipedia)]]
*/
class Prefixspan(
val sequences: RDD[Array[Int]],
val minSupport: Int = 2,
val maxPatternLength: Int = 50) extends java.io.Serializable {

/**
* Calculate sequential patterns:
* a) find and collect length-one patterns
* b) for each length-one patterns and each sequence,
* emit (pattern (prefix), suffix sequence) as key-value pairs
* c) group by key and then map value iterator to array
* d) local PrefixSpan on each prefix
* @return sequential patterns
*/
def run(): RDD[(Seq[Int], Int)] = {
val (patternsOneLength, prefixAndCandidates) = findPatternsLengthOne()
val repartitionedRdd = repartitionSequences(prefixAndCandidates)
val nextPatterns = getPatternsInLocal(repartitionedRdd)
val allPatterns = patternsOneLength.map(x => (Seq(x._1), x._2)) ++ nextPatterns
allPatterns
}

/**
* Find the patterns that it's length is one
* @return length-one patterns and projection table
*/
private def findPatternsLengthOne(): (RDD[(Int, Int)], RDD[(Seq[Int], Array[Int])]) = {
val patternsOneLength = sequences
.map(_.distinct)
.flatMap(p => p)
.map((_, 1))
.reduceByKey(_ + _)

val removedElements: Array[Int] = patternsOneLength
.filter(_._2 < minSupport)
.map(_._1)
.collect()

val savedElements = patternsOneLength.filter(_._2 >= minSupport)

val savedElementsArray = savedElements
.map(_._1)
.collect()

val filteredSequences =
if (removedElements.isEmpty) {
sequences
} else {
sequences.map { p =>
p.filter { x => !removedElements.contains(x) }
}
}

val prefixAndCandidates = filteredSequences.flatMap { x =>
savedElementsArray.map { y =>
val sub = getSuffix(y, x)
(Seq(y), sub)
}
}

(savedElements, prefixAndCandidates)
}

/**
* Re-partition the RDD data, to get better balance and performance.
* @param data patterns and projected sequences data before re-partition
* @return patterns and projected sequences data after re-partition
*/
private def repartitionSequences(
data: RDD[(Seq[Int], Array[Int])]): RDD[(Seq[Int], Array[Array[Int]])] = {
val dataRemovedEmptyLine = data.filter(x => x._2.nonEmpty)
val dataMerged = dataRemovedEmptyLine
.groupByKey()
.map(x => (x._1, x._2.toArray))
dataMerged
}

/**
* calculate the patterns in local.
* @param data patterns and projected sequences data data
* @return patterns
*/
private def getPatternsInLocal(
data: RDD[(Seq[Int], Array[Array[Int]])]): RDD[(Seq[Int], Int)] = {
val result = data.flatMap { x =>
getPatternsWithPrefix(x._1, x._2)
}
result
}

/**
* calculate the patterns with one prefix in local.
* @param prefix prefix
* @param data patterns and projected sequences data
* @return patterns
*/
private def getPatternsWithPrefix(
prefix: Seq[Int],
data: Array[Array[Int]]): Array[(Seq[Int], Int)] = {
val elements = data
.map(x => x.distinct)
.flatMap(x => x)
.groupBy(x => x)
.map(x => (x._1, x._2.length))

val selectedSingleElements = elements.filter(x => x._2 >= minSupport)

val selectedElements = selectedSingleElements
.map(x => (prefix ++ Seq(x._1), x._2))
.toArray

val cleanedSearchSpace = data
.map(x => x.filter(y => selectedSingleElements.contains(y)))

val newSearchSpace = selectedSingleElements.map { x =>
val sub = cleanedSearchSpace.map(y => getSuffix(x._1, y)).filter(_.nonEmpty)
(prefix ++ Seq(x._1), sub)
}.filter(x => x._2.nonEmpty)
.toArray

val continueProcess = newSearchSpace.nonEmpty && prefix.length + 1 < maxPatternLength

if (continueProcess) {
val nextPatterns = newSearchSpace
.map(x => getPatternsWithPrefix(x._1, x._2))
.reduce(_ ++ _)
selectedElements ++ nextPatterns
} else {
selectedElements
}
}

/**
* calculate suffix sequence following a prefix in a sequence
* @param prefix prefix
* @param sequence original sequence
* @return suffix sequence
*/
private def getSuffix(prefix: Int, sequence: Array[Int]): Array[Int] = {
val index = sequence.indexOf(prefix)
if (index == -1) {
Array()
} else {
sequence.takeRight(sequence.length - index - 1)
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.mllib.fpm

import org.apache.spark.SparkFunSuite
import org.apache.spark.mllib.util.MLlibTestSparkContext

class PrefixspanSuite extends SparkFunSuite with MLlibTestSparkContext {

test("Prefixspan sequences mining using Integer type") {
val sequences = Array(
Array(3, 1, 3, 4, 5),
Array(2, 3, 1),
Array(3, 4, 4, 3),
Array(1, 3, 4, 5),
Array(2, 4, 1),
Array(6, 5, 3))

val rdd = sc.parallelize(sequences, 2).cache()

val prefixspan1 = new Prefixspan(rdd, 2, 50)
val result1 = prefixspan1.run()
assert(result1.count() == 19)

val prefixspan2 = new Prefixspan(rdd, 3, 50)
val result2 = prefixspan2.run()
assert(result2.count() == 5)

val prefixspan3 = new Prefixspan(rdd, 2, 2)
val result3 = prefixspan3.run()
assert(result3.count() == 14)
}
}