@@ -45,7 +45,11 @@ class PrefixSpan private (
45
45
private var minSupport : Double ,
46
46
private var maxPatternLength : Int ) extends Logging with Serializable {
47
47
48
- private val maxProjectedDBSizeBeforeLocalProcessing : Long = 10000
48
+ /**
49
+ * The maximum number of items allowed in a projected database before local processing. If a
50
+ * projected database exceeds this size, another iteration of distributed PrefixSpan is run.
51
+ */
52
+ private val maxLocalProjDBSize : Long = 10000
49
53
50
54
/**
51
55
* Constructs a default instance with default parameters
@@ -63,8 +67,7 @@ class PrefixSpan private (
63
67
* Sets the minimal support level (default: `0.1`).
64
68
*/
65
69
def setMinSupport (minSupport : Double ): this .type = {
66
- require(minSupport >= 0 && minSupport <= 1 ,
67
- " The minimum support value must be in [0, 1]." )
70
+ require(minSupport >= 0 && minSupport <= 1 , " The minimum support value must be in [0, 1]." )
68
71
this .minSupport = minSupport
69
72
this
70
73
}
@@ -79,8 +82,7 @@ class PrefixSpan private (
79
82
*/
80
83
def setMaxPatternLength (maxPatternLength : Int ): this .type = {
81
84
// TODO: support unbounded pattern length when maxPatternLength = 0
82
- require(maxPatternLength >= 1 ,
83
- " The maximum pattern length value must be greater than 0." )
85
+ require(maxPatternLength >= 1 , " The maximum pattern length value must be greater than 0." )
84
86
this .maxPatternLength = maxPatternLength
85
87
this
86
88
}
@@ -119,13 +121,13 @@ class PrefixSpan private (
119
121
}.filter(_._2.nonEmpty)
120
122
}
121
123
}
122
- var (smallPrefixSuffixPairs, largePrefixSuffixPairs) = splitPrefixSuffixPairs (prefixSuffixPairs)
124
+ var (smallPrefixSuffixPairs, largePrefixSuffixPairs) = partitionByProjDBSize (prefixSuffixPairs)
123
125
124
126
while (largePrefixSuffixPairs.count() != 0 ) {
125
127
val (nextPatternAndCounts, nextPrefixSuffixPairs) =
126
128
getPatternCountsAndPrefixSuffixPairs(minCount, largePrefixSuffixPairs)
127
129
largePrefixSuffixPairs.unpersist()
128
- val (smallerPairsPart, largerPairsPart) = splitPrefixSuffixPairs (nextPrefixSuffixPairs)
130
+ val (smallerPairsPart, largerPairsPart) = partitionByProjDBSize (nextPrefixSuffixPairs)
129
131
largePrefixSuffixPairs = largerPairsPart
130
132
largePrefixSuffixPairs.persist(StorageLevel .MEMORY_AND_DISK )
131
133
smallPrefixSuffixPairs ++= smallerPairsPart
@@ -136,7 +138,6 @@ class PrefixSpan private (
136
138
val projectedDatabase = smallPrefixSuffixPairs
137
139
// TODO aggregateByKey
138
140
.groupByKey()
139
- .mapValues(_.toArray)
140
141
val nextPatternAndCounts = getPatternsInLocal(minCount, projectedDatabase)
141
142
allPatternAndCounts ++= nextPatternAndCounts
142
143
}
@@ -145,23 +146,21 @@ class PrefixSpan private (
145
146
146
147
147
148
/**
148
- * Split prefix suffix pairs to two parts:
149
- * Prefixes with projected databases smaller than maxSuffixesBeforeLocalProcessing and
150
- * Prefixes with projected databases larger than maxSuffixesBeforeLocalProcessing
149
+ * Partitions the prefix-suffix pairs by projected database size.
150
+ *
151
151
* @param prefixSuffixPairs prefix (length n) and suffix pairs,
152
- * @return small size prefix suffix pairs and big size prefix suffix pairs
153
- * (RDD[prefix, suffix], RDD[prefix, suffix ])
152
+ * @return prefix- suffix pairs partitioned by whether their projected database size is <= or
153
+ * greater than [[ maxLocalProjDBSize ]]
154
154
*/
155
- private def splitPrefixSuffixPairs (
156
- prefixSuffixPairs : RDD [(List [Int ], Array [Int ])]):
157
- (RDD [(List [Int ], Array [Int ])], RDD [(List [Int ], Array [Int ])]) = {
155
+ private def partitionByProjDBSize (prefixSuffixPairs : RDD [(List [Int ], Array [Int ])])
156
+ : (RDD [(List [Int ], Array [Int ])], RDD [(List [Int ], Array [Int ])]) = {
158
157
val prefixToSuffixSize = prefixSuffixPairs
159
158
.aggregateByKey(0 )(
160
159
seqOp = { case (count, suffix) => count + suffix.length },
161
160
combOp = { _ + _ })
162
161
val smallPrefixes = prefixToSuffixSize
163
- .filter(_._2 <= maxProjectedDBSizeBeforeLocalProcessing )
164
- .map(_._1)
162
+ .filter(_._2 <= maxLocalProjDBSize )
163
+ .keys
165
164
.collect()
166
165
.toSet
167
166
val small = prefixSuffixPairs.filter { case (prefix, _) => smallPrefixes.contains(prefix) }
@@ -214,7 +213,7 @@ class PrefixSpan private (
214
213
*/
215
214
private def getPatternsInLocal (
216
215
minCount : Long ,
217
- data : RDD [(List [Int ], Array [Array [Int ]])]): RDD [(List [Int ], Long )] = {
216
+ data : RDD [(List [Int ], Iterable [Array [Int ]])]): RDD [(List [Int ], Long )] = {
218
217
data.flatMap {
219
218
case (prefix, projDB) =>
220
219
LocalPrefixSpan .run(minCount, maxPatternLength, prefix.toList.reverse, projDB)
0 commit comments