2015-04-09 18:10:10 -04:00
|
|
|
#
|
|
|
|
# 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.
|
|
|
|
#
|
|
|
|
|
2018-03-08 06:38:34 -05:00
|
|
|
import sys
|
|
|
|
|
2015-04-22 20:22:26 -04:00
|
|
|
from collections import namedtuple
|
|
|
|
|
2019-01-17 20:40:39 -05:00
|
|
|
from pyspark import since
|
2016-02-25 02:22:14 -05:00
|
|
|
from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc
|
|
|
|
from pyspark.mllib.util import JavaSaveable, JavaLoader, inherit_doc
|
2015-04-09 18:10:10 -04:00
|
|
|
|
2015-11-04 18:28:19 -05:00
|
|
|
__all__ = ['FPGrowth', 'FPGrowthModel', 'PrefixSpan', 'PrefixSpanModel']
|
2015-04-09 18:10:10 -04:00
|
|
|
|
|
|
|
|
|
|
|
@inherit_doc
|
2016-02-25 02:22:14 -05:00
|
|
|
class FPGrowthModel(JavaModelWrapper, JavaSaveable, JavaLoader):
|
2015-04-09 18:10:10 -04:00
|
|
|
"""
|
|
|
|
A FP-Growth model for mining frequent itemsets
|
|
|
|
using the Parallel FP-Growth algorithm.
|
|
|
|
|
|
|
|
>>> data = [["a", "b", "c"], ["a", "b", "d", "e"], ["a", "c", "e"], ["a", "c", "f"]]
|
|
|
|
>>> rdd = sc.parallelize(data, 2)
|
|
|
|
>>> model = FPGrowth.train(rdd, 0.6, 2)
|
2015-07-06 19:11:22 -04:00
|
|
|
>>> sorted(model.freqItemsets().collect())
|
2020-07-13 22:22:44 -04:00
|
|
|
[FreqItemset(items=['a'], freq=4), FreqItemset(items=['c'], freq=3), ...
|
2016-02-25 02:22:14 -05:00
|
|
|
>>> model_path = temp_path + "/fpm"
|
|
|
|
>>> model.save(sc, model_path)
|
|
|
|
>>> sameModel = FPGrowthModel.load(sc, model_path)
|
|
|
|
>>> sorted(model.freqItemsets().collect()) == sorted(sameModel.freqItemsets().collect())
|
|
|
|
True
|
2015-09-17 11:50:46 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.4.0
|
2015-04-09 18:10:10 -04:00
|
|
|
"""
|
|
|
|
|
2015-09-17 11:50:46 -04:00
|
|
|
@since("1.4.0")
|
2015-04-09 18:10:10 -04:00
|
|
|
def freqItemsets(self):
|
|
|
|
"""
|
2015-04-22 20:22:26 -04:00
|
|
|
Returns the frequent itemsets of this model.
|
2015-04-09 18:10:10 -04:00
|
|
|
"""
|
2015-04-22 20:22:26 -04:00
|
|
|
return self.call("getFreqItemsets").map(lambda x: (FPGrowth.FreqItemset(x[0], x[1])))
|
2015-04-09 18:10:10 -04:00
|
|
|
|
2016-02-25 02:22:14 -05:00
|
|
|
@classmethod
|
|
|
|
@since("2.0.0")
|
|
|
|
def load(cls, sc, path):
|
|
|
|
"""
|
|
|
|
Load a model from the given path.
|
|
|
|
"""
|
|
|
|
model = cls._load_java(sc, path)
|
2016-07-05 20:00:24 -04:00
|
|
|
wrapper = sc._jvm.org.apache.spark.mllib.api.python.FPGrowthModelWrapper(model)
|
2016-02-25 02:22:14 -05:00
|
|
|
return FPGrowthModel(wrapper)
|
|
|
|
|
2015-04-09 18:10:10 -04:00
|
|
|
|
|
|
|
class FPGrowth(object):
|
|
|
|
"""
|
|
|
|
A Parallel FP-growth algorithm to mine frequent itemsets.
|
2015-09-17 11:50:46 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.4.0
|
2015-04-09 18:10:10 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
@classmethod
|
2015-09-17 11:50:46 -04:00
|
|
|
@since("1.4.0")
|
2015-04-09 18:10:10 -04:00
|
|
|
def train(cls, data, minSupport=0.3, numPartitions=-1):
|
|
|
|
"""
|
|
|
|
Computes an FP-Growth model that contains frequent itemsets.
|
2015-05-18 11:35:14 -04:00
|
|
|
|
2016-02-22 05:48:37 -05:00
|
|
|
:param data:
|
|
|
|
The input data set, each element contains a transaction.
|
|
|
|
:param minSupport:
|
|
|
|
The minimal support level.
|
|
|
|
(default: 0.3)
|
|
|
|
:param numPartitions:
|
|
|
|
The number of partitions used by parallel FP-growth. A value
|
|
|
|
of -1 will use the same number as input data.
|
|
|
|
(default: -1)
|
2015-04-09 18:10:10 -04:00
|
|
|
"""
|
|
|
|
model = callMLlibFunc("trainFPGrowthModel", data, float(minSupport), int(numPartitions))
|
|
|
|
return FPGrowthModel(model)
|
|
|
|
|
2015-04-22 20:22:26 -04:00
|
|
|
class FreqItemset(namedtuple("FreqItemset", ["items", "freq"])):
|
|
|
|
"""
|
|
|
|
Represents an (items, freq) tuple.
|
2015-09-17 11:50:46 -04:00
|
|
|
|
|
|
|
.. versionadded:: 1.4.0
|
2015-04-22 20:22:26 -04:00
|
|
|
"""
|
|
|
|
|
2015-04-09 18:10:10 -04:00
|
|
|
|
2015-11-04 18:28:19 -05:00
|
|
|
@inherit_doc
|
|
|
|
class PrefixSpanModel(JavaModelWrapper):
|
|
|
|
"""
|
|
|
|
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())
|
2020-07-13 22:22:44 -04:00
|
|
|
[FreqSequence(sequence=[['a']], freq=3), FreqSequence(sequence=[['a'], ['a']], freq=1), ...
|
2015-11-04 18:28:19 -05:00
|
|
|
|
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
"""
|
|
|
|
|
|
|
|
@since("1.6.0")
|
|
|
|
def freqSequences(self):
|
2016-03-03 17:42:12 -05:00
|
|
|
"""Gets frequent sequences"""
|
2015-11-04 18:28:19 -05:00
|
|
|
return self.call("getFreqSequences").map(lambda x: PrefixSpan.FreqSequence(x[0], x[1]))
|
|
|
|
|
|
|
|
|
|
|
|
class PrefixSpan(object):
|
|
|
|
"""
|
|
|
|
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
|
2018-11-25 18:43:55 -05:00
|
|
|
([[https://doi.org/10.1109/ICDE.2001.914830]]).
|
2015-11-04 18:28:19 -05:00
|
|
|
|
|
|
|
.. versionadded:: 1.6.0
|
|
|
|
"""
|
|
|
|
|
|
|
|
@classmethod
|
|
|
|
@since("1.6.0")
|
|
|
|
def train(cls, data, minSupport=0.1, maxPatternLength=10, maxLocalProjDBSize=32000000):
|
|
|
|
"""
|
2016-02-22 05:48:37 -05:00
|
|
|
Finds the complete set of frequent sequential patterns in the
|
|
|
|
input sequences of itemsets.
|
|
|
|
|
|
|
|
:param data:
|
|
|
|
The input data set, each element contains a sequence of
|
|
|
|
itemsets.
|
|
|
|
: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)
|
|
|
|
: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)
|
2015-11-04 18:28:19 -05:00
|
|
|
"""
|
|
|
|
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
|
|
|
|
"""
|
|
|
|
|
|
|
|
|
2015-04-09 18:10:10 -04:00
|
|
|
def _test():
|
|
|
|
import doctest
|
2016-05-23 21:14:48 -04:00
|
|
|
from pyspark.sql import SparkSession
|
2015-04-09 18:10:10 -04:00
|
|
|
import pyspark.mllib.fpm
|
|
|
|
globs = pyspark.mllib.fpm.__dict__.copy()
|
2016-05-23 21:14:48 -04:00
|
|
|
spark = SparkSession.builder\
|
|
|
|
.master("local[4]")\
|
|
|
|
.appName("mllib.fpm tests")\
|
|
|
|
.getOrCreate()
|
|
|
|
globs['sc'] = spark.sparkContext
|
2016-02-25 02:22:14 -05:00
|
|
|
import tempfile
|
|
|
|
|
|
|
|
temp_path = tempfile.mkdtemp()
|
|
|
|
globs['temp_path'] = temp_path
|
|
|
|
try:
|
|
|
|
(failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
|
2016-05-23 21:14:48 -04:00
|
|
|
spark.stop()
|
2016-02-25 02:22:14 -05:00
|
|
|
finally:
|
|
|
|
from shutil import rmtree
|
|
|
|
try:
|
|
|
|
rmtree(temp_path)
|
|
|
|
except OSError:
|
|
|
|
pass
|
2015-04-09 18:10:10 -04:00
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2015-04-09 18:10:10 -04:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|