@@ -45,7 +45,7 @@ class PrefixSpan private (
45
45
private var minSupport : Double ,
46
46
private var maxPatternLength : Int ) extends Logging with Serializable {
47
47
48
- private val maxSuffixesBeforeLocalProcessing : Long = 10000
48
+ private val maxProjectedDBSizeBeforeLocalProcessing : Long = 10000
49
49
50
50
/**
51
51
* Constructs a default instance with default parameters
@@ -89,41 +89,36 @@ class PrefixSpan private (
89
89
val lengthOnePatternsAndCounts = getFreqItemAndCounts(minCount, sequences)
90
90
val prefixSuffixPairs = getPrefixSuffixPairs(
91
91
lengthOnePatternsAndCounts.map(_._1).collect(), sequences)
92
- var patternsCount : Long = lengthOnePatternsAndCounts.count( )
92
+ prefixSuffixPairs.persist( StorageLevel . MEMORY_AND_DISK )
93
93
var allPatternAndCounts = lengthOnePatternsAndCounts.map(x => (ArrayBuffer (x._1), x._2))
94
94
var (smallPrefixSuffixPairs, largePrefixSuffixPairs) =
95
95
splitPrefixSuffixPairs(prefixSuffixPairs)
96
- largePrefixSuffixPairs.persist(StorageLevel .MEMORY_AND_DISK )
97
- var patternLength : Int = 1
98
- while (patternLength < maxPatternLength &&
99
- largePrefixSuffixPairs.count() != 0 ) {
96
+ while (largePrefixSuffixPairs.count() != 0 ) {
100
97
val (nextPatternAndCounts, nextPrefixSuffixPairs) =
101
98
getPatternCountsAndPrefixSuffixPairs(minCount, largePrefixSuffixPairs)
102
- patternsCount = nextPatternAndCounts.count()
103
99
largePrefixSuffixPairs.unpersist()
104
- val splitedPrefixSuffixPairs = splitPrefixSuffixPairs(nextPrefixSuffixPairs)
105
- largePrefixSuffixPairs = splitedPrefixSuffixPairs._2
100
+ val (smallerPairsPart, largerPairsPart) = splitPrefixSuffixPairs(nextPrefixSuffixPairs)
101
+ largePrefixSuffixPairs = largerPairsPart
106
102
largePrefixSuffixPairs.persist(StorageLevel .MEMORY_AND_DISK )
107
- smallPrefixSuffixPairs = smallPrefixSuffixPairs ++ splitedPrefixSuffixPairs._1
108
- allPatternAndCounts = allPatternAndCounts ++ nextPatternAndCounts
109
- patternLength = patternLength + 1
103
+ smallPrefixSuffixPairs ++= smallerPairsPart
104
+ allPatternAndCounts ++= nextPatternAndCounts
110
105
}
111
106
if (smallPrefixSuffixPairs.count() > 0 ) {
112
107
val projectedDatabase = smallPrefixSuffixPairs
113
108
.map(x => (x._1.toSeq, x._2))
114
109
.groupByKey()
115
110
.map(x => (x._1.toArray, x._2.toArray))
116
111
val nextPatternAndCounts = getPatternsInLocal(minCount, projectedDatabase)
117
- allPatternAndCounts = allPatternAndCounts ++ nextPatternAndCounts
112
+ allPatternAndCounts ++= nextPatternAndCounts
118
113
}
119
114
allPatternAndCounts.map { case (pattern, count) => (pattern.toArray, count) }
120
115
}
121
116
122
117
123
118
/**
124
119
* Split prefix suffix pairs to two parts:
125
- * suffixes' size less than maxSuffixesBeforeLocalProcessing and
126
- * suffixes' size more than maxSuffixesBeforeLocalProcessing
120
+ * Prefixes with projected databases smaller than maxSuffixesBeforeLocalProcessing and
121
+ * Prefixes with projected databases larger than maxSuffixesBeforeLocalProcessing
127
122
* @param prefixSuffixPairs prefix (length n) and suffix pairs,
128
123
* @return small size prefix suffix pairs and big size prefix suffix pairs
129
124
* (RDD[prefix, suffix], RDD[prefix, suffix ])
@@ -134,7 +129,7 @@ class PrefixSpan private (
134
129
val suffixSizeMap = prefixSuffixPairs
135
130
.map(x => (x._1, x._2.length))
136
131
.reduceByKey(_ + _)
137
- .map(x => (x._2 <= maxSuffixesBeforeLocalProcessing , Set (x._1)))
132
+ .map(x => (x._2 <= maxProjectedDBSizeBeforeLocalProcessing , Set (x._1)))
138
133
.reduceByKey(_ ++ _)
139
134
.collect
140
135
.toMap
0 commit comments