Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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
45 changes: 25 additions & 20 deletions mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,7 @@ import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.mllib.fpm.{PrefixSpan => mllibPrefixSpan}
import org.apache.spark.sql.{DataFrame, Dataset, Row}
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.types.{LongType, StructField, StructType}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.sql.types.{ArrayType, LongType, StructField, StructType}

/**
* :: Experimental ::
Expand All @@ -44,26 +43,37 @@ object PrefixSpan {
*
* @param dataset A dataset or a dataframe containing a sequence column which is
* {{{Seq[Seq[_]]}}} type
* @param sequenceCol the name of the sequence column in dataset
* @param sequenceCol the name of the sequence column in dataset, rows with nulls in this column
* are ignored
* @param minSupport the minimal support level of the sequential pattern, any pattern that
* appears more than (minSupport * size-of-the-dataset) times will be output
* (default: `0.1`).
* @param maxPatternLength the maximal length of the sequential pattern, any pattern that appears
* less than maxPatternLength will be output (default: `10`).
* (recommended value: `0.1`).
* @param maxPatternLength the maximal length of the sequential pattern
* (recommended value: `10`).
* @param maxLocalProjDBSize The maximum number of items (including delimiters used in the
* internal storage format) allowed in a projected database before
* local processing. If a projected database exceeds this size, another
* iteration of distributed prefix growth is run (default: `32000000`).
* @return A dataframe that contains columns of sequence and corresponding frequency.
* iteration of distributed prefix growth is run
* (recommended value: `32000000`).
* @return A `DataFrame` that contains columns of sequence and corresponding frequency.
* The schema of it will be:
* - `sequence: Seq[Seq[T]]` (T is the item type)
* - `frequency: Long`
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had asked for this change to "frequency" from "freq," but I belatedly realized that this conflicts with the existing FPGrowth API, which uses "freq." It would be best to maintain consistency. Would you mind reverting to "freq?"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure!

*/
@Since("2.4.0")
def findFrequentSequentPatterns(
def findFrequentSequentialPatterns(
dataset: Dataset[_],
sequenceCol: String,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@WeichenXu123 @jkbradley The static method doesn't scale with parameters. If we add a new param, we have to keep the old one for binary compatibility. Why not using setters? I think we only need to avoid using fit and transform names.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with using setters. @jkbradley What do you think of it ?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree in general, but I don’t think it’s a big deal for PrefixSpan. I think of our current static method as a temporary workaround until we do the work to build a Model which can make meaningful predictions. This will mean that further PrefixSpan improvements may be blocked on this Model work, but I think that’s OK since predictions should be the next priority for PrefixSpan. Once we have a Model, I recommend we deprecate the current static method.

I'm also OK with changing this to use setters, but then we should name it something else so that we can replace it with an Estimator + Model pair later on. I'd suggest "PrefixSpanBuilder."

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It should be easier to keep the PrefixSpan name and make it an Estimator later. For example:

final class PrefixSpan(override val uid: String) extends Params {
  // param, setters, getters
  def findFrequentSequentialPatterns(dataset: Dataset[_]): DataFrame
}

Later we can add Estimator.fit and PrefixSpanModel.transform. Any issue with this approach?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this way final class PrefixSpan(override val uid: String) extends Params seemingly breaks binary compatibility if later we change it into an estimator ?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Adding extends Estimator later should only introduce new methods to the class but no breaking changes.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I think you're right @mengxr . That approach sounds good.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@WeichenXu123 Do you have time to send a PR to update this API?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure. Will update soon!

minSupport: Double = 0.1,
maxPatternLength: Int = 10,
maxLocalProjDBSize: Long = 32000000L): DataFrame = {
val handlePersistence = dataset.storageLevel == StorageLevel.NONE
minSupport: Double,
maxPatternLength: Int,
maxLocalProjDBSize: Long): DataFrame = {

val inputType = dataset.schema(sequenceCol).dataType
require(inputType.isInstanceOf[ArrayType] &&
inputType.asInstanceOf[ArrayType].elementType.isInstanceOf[ArrayType],
s"The input column must be ArrayType and the array element type must also be ArrayType, " +
s"but got $inputType.")


val data = dataset.select(sequenceCol)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's check the input schema and throw a clear exception if it's not OK.

val sequences = data.where(col(sequenceCol).isNotNull).rdd
Expand All @@ -73,18 +83,13 @@ object PrefixSpan {
.setMinSupport(minSupport)
.setMaxPatternLength(maxPatternLength)
.setMaxLocalProjDBSize(maxLocalProjDBSize)
if (handlePersistence) {
sequences.persist(StorageLevel.MEMORY_AND_DISK)
}

val rows = mllibPrefixSpan.run(sequences).freqSequences.map(f => Row(f.sequence, f.freq))
val schema = StructType(Seq(
StructField("sequence", dataset.schema(sequenceCol).dataType, nullable = false),
StructField("freq", LongType, nullable = false)))
StructField("frequency", LongType, nullable = false)))
val freqSequences = dataset.sparkSession.createDataFrame(rows, schema)

if (handlePersistence) {
sequences.unpersist()
}
freqSequences
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,7 @@ import org.apache.spark.storage.StorageLevel
*
* @param minSupport the minimal support level of the sequential pattern, any pattern that appears
* more than (minSupport * size-of-the-dataset) times will be output
* @param maxPatternLength the maximal length of the sequential pattern, any pattern that appears
* less than maxPatternLength will be output
* @param maxPatternLength the maximal length of the sequential pattern
* @param maxLocalProjDBSize The maximum number of items (including delimiters used in the internal
* storage format) allowed in a projected database before local
* processing. If a projected database exceeds this size, another
Expand Down
54 changes: 45 additions & 9 deletions mllib/src/test/scala/org/apache/spark/ml/fpm/PrefixSpanSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,13 @@ class PrefixSpanSuite extends MLTest {

override def beforeAll(): Unit = {
super.beforeAll()
smallDataset = Seq(Seq(Seq(1, 2), Seq(1, 2, 3))).toDF("sequence")
}

@transient var smallDataset: DataFrame = _

test("PrefixSpan projections with multiple partial starts") {
val result = PrefixSpan.findFrequentSequentPatterns(smallDataset, "sequence",
minSupport = 1.0, maxPatternLength = 2).as[(Seq[Seq[Int]], Long)].collect()
val smallDataset = Seq(Seq(Seq(1, 2), Seq(1, 2, 3))).toDF("sequence")
val result = PrefixSpan.findFrequentSequentialPatterns(smallDataset, "sequence",
minSupport = 1.0, maxPatternLength = 2, maxLocalProjDBSize = 32000000)
.as[(Seq[Seq[Int]], Long)].collect()
val expected = Array(
(Seq(Seq(1)), 1L),
(Seq(Seq(1, 2)), 1L),
Expand All @@ -49,6 +48,32 @@ class PrefixSpanSuite extends MLTest {
compareResults[Int](expected, result)
}

/*
To verify expected results for `smallTestData`, create file "prefixSpanSeqs2" with content
(format = (transactionID, idxInTransaction, numItemsinItemset, itemset)):
1 1 2 1 2
1 2 1 3
2 1 1 1
2 2 2 3 2
2 3 2 1 2
3 1 2 1 2
3 2 1 5
4 1 1 6
In R, run:
library("arulesSequences")
prefixSpanSeqs = read_baskets("prefixSpanSeqs", info = c("sequenceID","eventID","SIZE"))
freqItemSeq = cspade(prefixSpanSeqs,
parameter = 0.5, maxlen = 5 ))
resSeq = as(freqItemSeq, "data.frame")
resSeq

sequence support
1 <{1}> 0.75
2 <{2}> 0.75
3 <{3}> 0.50
4 <{1},{3}> 0.50
5 <{1,2}> 0.75
*/
val smallTestData = Seq(
Seq(Seq(1, 2), Seq(3)),
Seq(Seq(1), Seq(3, 2), Seq(1, 2)),
Expand All @@ -65,8 +90,18 @@ class PrefixSpanSuite extends MLTest {

test("PrefixSpan Integer type, variable-size itemsets") {
val df = smallTestData.toDF("sequence")
val result = PrefixSpan.findFrequentSequentPatterns(df, "sequence",
minSupport = 0.5, maxPatternLength = 5).as[(Seq[Seq[Int]], Long)].collect()
val result = PrefixSpan.findFrequentSequentialPatterns(df, "sequence",
minSupport = 0.5, maxPatternLength = 5, maxLocalProjDBSize = 32000000)
.as[(Seq[Seq[Int]], Long)].collect()

compareResults[Int](smallTestDataExpectedResult, result)
}

test("PrefixSpan input row with nulls") {
val df = (smallTestData :+ null).toDF("sequence")
val result = PrefixSpan.findFrequentSequentialPatterns(df, "sequence",
minSupport = 0.5, maxPatternLength = 5, maxLocalProjDBSize = 32000000)
.as[(Seq[Seq[Int]], Long)].collect()

compareResults[Int](smallTestDataExpectedResult, result)
}
Expand All @@ -76,8 +111,9 @@ class PrefixSpanSuite extends MLTest {
val df = smallTestData
.map(seq => seq.map(itemSet => itemSet.map(intToString)))
.toDF("sequence")
val result = PrefixSpan.findFrequentSequentPatterns(df, "sequence",
minSupport = 0.5, maxPatternLength = 5).as[(Seq[Seq[String]], Long)].collect()
val result = PrefixSpan.findFrequentSequentialPatterns(df, "sequence",
minSupport = 0.5, maxPatternLength = 5, maxLocalProjDBSize = 32000000)
.as[(Seq[Seq[String]], Long)].collect()

val expected = smallTestDataExpectedResult.map { case (seq, freq) =>
(seq.map(itemSet => itemSet.map(intToString)), freq)
Expand Down