@@ -91,9 +91,8 @@ class PrefixSpan private (
9191
9292 /**
9393 * Find the complete set of sequential patterns in the input sequences.
94- * @param sequences a dataset of sequences. Items in a sequence are represented by non-negative
95- * integers and delimited by [[DELIMITER ]]. Non-temporal sequences
96- * are supported by placing more than one item between delimiters.
94+ * @param sequences ordered sequences of itemsets. Items are represented by non-negative integers.
95+ * Each itemset has one or more items and is delimited by [[DELIMITER ]].
9796 * @return a set of sequential pattern pairs,
9897 * the key of pair is pattern (a list of elements),
9998 * the value of pair is the pattern's count.
@@ -124,22 +123,23 @@ class PrefixSpan private (
124123 freqItems.flatMap { item =>
125124 val candidateSuffix = LocalPrefixSpan .getSuffix(item, filteredSeq)
126125 candidateSuffix match {
127- case suffix if ! suffix.isEmpty => Some ((List (item), suffix))
126+ case suffix if ! suffix.isEmpty => Some ((List (DELIMITER , item), suffix))
128127 case _ => None
129128 }
130129 }
131130 }
132131 }
133132 // Accumulator for the computed results to be returned, initialized to the frequent items (i.e.
134133 // frequent length-one prefixes)
135- var resultsAccumulator = freqItemCounts.map(x => (List (x._1), x._2))
134+ var resultsAccumulator = freqItemCounts.map(x => (List (DELIMITER , x._1), x._2))
136135
137136 // Remaining work to be locally and distributively processed respectfully
138137 var (pairsForLocal, pairsForDistributed) = partitionByProjDBSize(itemSuffixPairs)
139138
140139 // Continue processing until no pairs for distributed processing remain (i.e. all prefixes have
141- // projected database sizes <= `maxLocalProjDBSize`)
142- while (pairsForDistributed.count() != 0 ) {
140+ // projected database sizes <= `maxLocalProjDBSize`) or `maxPatternLength` is reached
141+ var patternLength = 1
142+ while (pairsForDistributed.count() != 0 || patternLength < maxPatternLength) {
143143 val (nextPatternAndCounts, nextPrefixSuffixPairs) =
144144 extendPrefixes(minCount, pairsForDistributed)
145145 pairsForDistributed.unpersist()
@@ -148,14 +148,15 @@ class PrefixSpan private (
148148 pairsForDistributed.persist(StorageLevel .MEMORY_AND_DISK )
149149 pairsForLocal ++= smallerPairsPart
150150 resultsAccumulator ++= nextPatternAndCounts.collect()
151+ patternLength += 1 // pattern length grows one per iteration
151152 }
152153
153154 // Process the small projected databases locally
154155 val remainingResults = getPatternsInLocal(
155156 minCount, sc.parallelize(pairsForLocal, 1 ).groupByKey())
156157
157158 (sc.parallelize(resultsAccumulator, 1 ) ++ remainingResults)
158- .map { case (pattern, count) => (pattern.toArray, count) }
159+ .map { case (pattern, count) => (pattern.reverse. toArray, count) }
159160 }
160161
161162
@@ -209,10 +210,9 @@ class PrefixSpan private (
209210 .collect()
210211 .toMap
211212
212-
213213 // Frequent patterns with length N+1 and their corresponding counts
214214 val extendedPrefixAndCounts = prefixItemPairAndCounts
215- .map { case ((prefix, item), count) => (item :: prefix, count) }
215+ .map { case ((prefix, item), count) => (DELIMITER :: item :: prefix, count) }
216216
217217 // Remaining work, all prefixes will have length N+1
218218 val extendedPrefixAndSuffix = prefixSuffixPairs
@@ -222,7 +222,7 @@ class PrefixSpan private (
222222 val filteredSuffix = suffix.filter(frequentNextItems.contains(_))
223223 frequentNextItems.flatMap { item =>
224224 LocalPrefixSpan .getSuffix(item, filteredSuffix) match {
225- case suffix if ! suffix.isEmpty => Some (item :: prefix, suffix)
225+ case suffix if ! suffix.isEmpty => Some (DELIMITER :: item :: prefix, suffix)
226226 case _ => None
227227 }
228228 }
@@ -242,9 +242,9 @@ class PrefixSpan private (
242242 data : RDD [(List [Int ], Iterable [Array [Int ]])]): RDD [(List [Int ], Long )] = {
243243 data.flatMap {
244244 case (prefix, projDB) =>
245- LocalPrefixSpan .run(minCount, maxPatternLength, prefix.toList.reverse , projDB)
245+ LocalPrefixSpan .run(minCount, maxPatternLength, prefix.toList, projDB)
246246 .map { case (pattern : List [Int ], count : Long ) =>
247- (pattern.reverse , count)
247+ (pattern, count)
248248 }
249249 }
250250 }
0 commit comments