Skip to content

Commit 6e149fa

Browse files
author
Feynman Liang
committed
Fix splitPrefixSuffixPairs
1 parent 64271b3 commit 6e149fa

File tree

2 files changed

+23
-28
lines changed

2 files changed

+23
-28
lines changed

mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala

Lines changed: 12 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ class PrefixSpan private (
5858
*/
5959
def setMinSupport(minSupport: Double): this.type = {
6060
require(minSupport >= 0 && minSupport <= 1,
61-
"The minimum support value must be between 0 and 1, including 0 and 1.")
61+
"The minimum support value must be in [0, 1].")
6262
this.minSupport = minSupport
6363
this
6464
}
@@ -126,23 +126,17 @@ class PrefixSpan private (
126126
private def splitPrefixSuffixPairs(
127127
prefixSuffixPairs: RDD[(ArrayBuffer[Int], Array[Int])]):
128128
(RDD[(ArrayBuffer[Int], Array[Int])], RDD[(ArrayBuffer[Int], Array[Int])]) = {
129-
val suffixSizeMap = prefixSuffixPairs
130-
.map(x => (x._1, x._2.length))
131-
.reduceByKey(_ + _)
132-
.map(x => (x._2 <= maxProjectedDBSizeBeforeLocalProcessing, Set(x._1)))
133-
.reduceByKey(_ ++ _)
134-
.collect
135-
.toMap
136-
val small = if (suffixSizeMap.contains(true)) {
137-
prefixSuffixPairs.filter(x => suffixSizeMap(true).contains(x._1))
138-
} else {
139-
prefixSuffixPairs.filter(x => false)
140-
}
141-
val large = if (suffixSizeMap.contains(false)) {
142-
prefixSuffixPairs.filter(x => suffixSizeMap(false).contains(x._1))
143-
} else {
144-
prefixSuffixPairs.filter(x => false)
145-
}
129+
val prefixToSuffixSize = prefixSuffixPairs
130+
.aggregateByKey(0)(
131+
seqOp = { case (count, suffix) => count + suffix.length },
132+
combOp = { _ + _ })
133+
val smallPrefixes = prefixToSuffixSize
134+
.filter(_._2 <= maxProjectedDBSizeBeforeLocalProcessing)
135+
.map(_._1)
136+
.collect()
137+
.toSet
138+
val small = prefixSuffixPairs.filter { case (prefix, _) => smallPrefixes.contains(prefix) }
139+
val large = prefixSuffixPairs.filter { case (prefix, _) => !smallPrefixes.contains(prefix) }
146140
(small, large)
147141
}
148142

mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -44,13 +44,6 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext {
4444

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

47-
def compareResult(
48-
expectedValue: Array[(Array[Int], Long)],
49-
actualValue: Array[(Array[Int], Long)]): Boolean = {
50-
expectedValue.map(x => (x._1.toSeq, x._2)).toSet ==
51-
actualValue.map(x => (x._1.toSeq, x._2)).toSet
52-
}
53-
5447
val prefixspan = new PrefixSpan()
5548
.setMinSupport(0.33)
5649
.setMaxPatternLength(50)
@@ -76,7 +69,7 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext {
7669
(Array(4, 5), 2L),
7770
(Array(5), 3L)
7871
)
79-
assert(compareResult(expectedValue1, result1.collect()))
72+
assert(compareResults(expectedValue1, result1.collect()))
8073

8174
prefixspan.setMinSupport(0.5).setMaxPatternLength(50)
8275
val result2 = prefixspan.run(rdd)
@@ -87,7 +80,7 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext {
8780
(Array(4), 4L),
8881
(Array(5), 3L)
8982
)
90-
assert(compareResult(expectedValue2, result2.collect()))
83+
assert(compareResults(expectedValue2, result2.collect()))
9184

9285
prefixspan.setMinSupport(0.33).setMaxPatternLength(2)
9386
val result3 = prefixspan.run(rdd)
@@ -107,6 +100,14 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext {
107100
(Array(4, 5), 2L),
108101
(Array(5), 3L)
109102
)
110-
assert(compareResult(expectedValue3, result3.collect()))
103+
assert(compareResults(expectedValue3, result3.collect()))
104+
}
105+
106+
private def compareResults(
107+
expectedValue: Array[(Array[Int], Long)],
108+
actualValue: Array[(Array[Int], Long)]): Boolean = {
109+
expectedValue.map(x => (x._1.toSeq, x._2)).toSet ==
110+
actualValue.map(x => (x._1.toSeq, x._2)).toSet
111111
}
112+
112113
}

0 commit comments

Comments
 (0)