[SPARK-10028][MLLIB][PYTHON] Add Python API for PrefixSpan

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #9469 from yu-iskw/SPARK-10028.
This commit is contained in:
Yu ISHIKAWA 2015-11-04 15:28:19 -08:00 committed by Xiangrui Meng
parent 1b6a5d4af9
commit 411ff6afb4
3 changed files with 122 additions and 2 deletions

View file

@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.mllib.api.python
import org.apache.spark.mllib.fpm.PrefixSpanModel
import org.apache.spark.rdd.RDD
/**
* A Wrapper of PrefixSpanModel to provide helper method for Python
*/
private[python] class PrefixSpanModelWrapper(model: PrefixSpanModel[Any])
extends PrefixSpanModel(model.freqSequences) {
def getFreqSequences: RDD[Array[Any]] = {
SerDe.fromTuple2RDD(model.freqSequences.map(x => (x.javaSequence, x.freq)))
}
}

View file

@ -35,7 +35,7 @@ import org.apache.spark.mllib.classification._
import org.apache.spark.mllib.clustering._
import org.apache.spark.mllib.evaluation.RankingMetrics
import org.apache.spark.mllib.feature._
import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel}
import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel, PrefixSpan}
import org.apache.spark.mllib.linalg._
import org.apache.spark.mllib.linalg.distributed._
import org.apache.spark.mllib.optimization._
@ -557,6 +557,27 @@ private[python] class PythonMLLibAPI extends Serializable {
new FPGrowthModelWrapper(model)
}
/**
* Java stub for Python mllib PrefixSpan.train(). This stub returns a handle
* to the Java object instead of the content of the Java object. Extra care
* needs to be taken in the Python code to ensure it gets freed on exit; see
* the Py4J documentation.
*/
def trainPrefixSpanModel(
data: JavaRDD[java.util.ArrayList[java.util.ArrayList[Any]]],
minSupport: Double,
maxPatternLength: Int,
localProjDBSize: Int ): PrefixSpanModelWrapper = {
val prefixSpan = new PrefixSpan()
.setMinSupport(minSupport)
.setMaxPatternLength(maxPatternLength)
.setMaxLocalProjDBSize(localProjDBSize)
val trainData = data.rdd.map(_.asScala.toArray.map(_.asScala.toArray))
val model = prefixSpan.run(trainData)
new PrefixSpanModelWrapper(model)
}
/**
* Java stub for Normalizer.transform()
*/

View file

@ -23,7 +23,7 @@ from pyspark import SparkContext, since
from pyspark.rdd import ignore_unicode_prefix
from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc
__all__ = ['FPGrowth', 'FPGrowthModel']
__all__ = ['FPGrowth', 'FPGrowthModel', 'PrefixSpan', 'PrefixSpanModel']
@inherit_doc
@ -85,6 +85,73 @@ class FPGrowth(object):
"""
@inherit_doc
@ignore_unicode_prefix
class PrefixSpanModel(JavaModelWrapper):
"""
.. note:: Experimental
Model fitted by PrefixSpan
>>> data = [
... [["a", "b"], ["c"]],
... [["a"], ["c", "b"], ["a", "b"]],
... [["a", "b"], ["e"]],
... [["f"]]]
>>> rdd = sc.parallelize(data, 2)
>>> model = PrefixSpan.train(rdd)
>>> sorted(model.freqSequences().collect())
[FreqSequence(sequence=[[u'a']], freq=3), FreqSequence(sequence=[[u'a'], [u'a']], freq=1), ...
.. versionadded:: 1.6.0
"""
@since("1.6.0")
def freqSequences(self):
"""Gets frequence sequences"""
return self.call("getFreqSequences").map(lambda x: PrefixSpan.FreqSequence(x[0], x[1]))
class PrefixSpan(object):
"""
.. note:: Experimental
A parallel PrefixSpan algorithm to mine frequent sequential patterns.
The PrefixSpan algorithm is described in J. Pei, et al., PrefixSpan:
Mining Sequential Patterns Efficiently by Prefix-Projected Pattern Growth
([[http://doi.org/10.1109/ICDE.2001.914830]]).
.. versionadded:: 1.6.0
"""
@classmethod
@since("1.6.0")
def train(cls, data, minSupport=0.1, maxPatternLength=10, maxLocalProjDBSize=32000000):
"""
Finds the complete set of frequent sequential patterns in the input sequences of itemsets.
:param data: The input data set, each element contains a sequnce of itemsets.
:param minSupport: the minimal support level of the sequential pattern, any pattern 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 appears
less than maxPatternLength will be output. (default: `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`)
"""
model = callMLlibFunc("trainPrefixSpanModel",
data, minSupport, maxPatternLength, maxLocalProjDBSize)
return PrefixSpanModel(model)
class FreqSequence(namedtuple("FreqSequence", ["sequence", "freq"])):
"""
Represents a (sequence, freq) tuple.
.. versionadded:: 1.6.0
"""
def _test():
import doctest
import pyspark.mllib.fpm