From 33b68e0625c5d14175b14813b197ce20caa3b466 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 22 Dec 2014 16:41:57 -0800 Subject: [PATCH 01/28] a working LR --- python/pyspark/ml/__init__.py | 22 ++++++++++ python/pyspark/ml/classification.py | 67 +++++++++++++++++++++++++++++ python/pyspark/ml/param.py | 22 ++++++++++ 3 files changed, 111 insertions(+) create mode 100644 python/pyspark/ml/__init__.py create mode 100644 python/pyspark/ml/classification.py create mode 100644 python/pyspark/ml/param.py diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py new file mode 100644 index 0000000000000..622acb96782e8 --- /dev/null +++ b/python/pyspark/ml/__init__.py @@ -0,0 +1,22 @@ +import inspect + +from pyspark import SparkContext + +# An implementation of PEP3102 for Python 2. +_keyword_only_secret = 70861589 + + +def _assert_keyword_only_args(): + """ + Checks whether the _keyword_only trick is applied and validates input arguments. + """ + # Get the frame of the function that calls this function. + frame = inspect.currentframe().f_back + info = inspect.getargvalues(frame) + if "_keyword_only" not in info.args: + raise ValueError("Function does not have argument _keyword_only.") + if info.locals["_keyword_only"] != _keyword_only_secret: + raise ValueError("Must use keyword arguments instead of positional ones.") + +def _jvm(): + return SparkContext._jvm diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py new file mode 100644 index 0000000000000..9fe81e864855f --- /dev/null +++ b/python/pyspark/ml/classification.py @@ -0,0 +1,67 @@ +from pyspark.sql import SchemaRDD +from pyspark.ml import _keyword_only_secret, _assert_keyword_only_args, _jvm +from pyspark.ml.param import Param + + +class LogisticRegression(object): + """ + Logistic regression. + """ + + _java_class = "org.apache.spark.ml.classification.LogisticRegression" + + def __init__(self): + self._java_obj = _jvm().org.apache.spark.ml.classification.LogisticRegression() + self.paramMap = {} + self.maxIter = Param(self, "maxIter", "max number of iterations", 100) + self.regParam = Param(self, "regParam", "regularization constant", 0.1) + + def set(self, _keyword_only=_keyword_only_secret, + maxIter=None, regParam=None): + _assert_keyword_only_args() + if maxIter is not None: + self.paramMap[self.maxIter] = maxIter + if regParam is not None: + self.paramMap[self.regParam] = regParam + return self + + # cannot chained + def setMaxIter(self, value): + self.paramMap[self.maxIter] = value + return self + + def setRegParam(self, value): + self.paramMap[self.regParam] = value + return self + + def getMaxIter(self): + if self.maxIter in self.paramMap: + return self.paramMap[self.maxIter] + else: + return self.maxIter.defaultValue + + def getRegParam(self): + if self.regParam in self.paramMap: + return self.paramMap[self.regParam] + else: + return self.regParam.defaultValue + + def fit(self, dataset): + java_model = self._java_obj.fit(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()) + return LogisticRegressionModel(java_model) + + +class LogisticRegressionModel(object): + """ + Model fitted by LogisticRegression. + """ + + def __init__(self, _java_model): + self._java_model = _java_model + + def transform(self, dataset): + return SchemaRDD(self._java_model.transform(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()), dataset.sql_ctx) + +lr = LogisticRegression() + +lr.set(maxIter=10, regParam=0.1) diff --git a/python/pyspark/ml/param.py b/python/pyspark/ml/param.py new file mode 100644 index 0000000000000..be85675233cde --- /dev/null +++ b/python/pyspark/ml/param.py @@ -0,0 +1,22 @@ +class Param(object): + """ + A param with self-contained documentation and optionally default value. + """ + + def __init__(self, parent, name, doc, defaultValue=None): + self.parent = parent + self.name = name + self.doc = doc + self.defaultValue = defaultValue + + def __str__(self): + return self.parent + "_" + self.name + + def __repr_(self): + return self.parent + "_" + self.name + + +class Params(object): + """ + Components that take parameters. + """ From 46eea43d25f18c58f2008e8fb083c6417a243ad6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 31 Dec 2014 11:33:59 -0800 Subject: [PATCH 02/28] a pipeline in python --- .../ml/simple_text_classification_pipeline.py | 33 +++++++ .../org/apache/spark/ml/param/params.scala | 7 ++ python/pyspark/ml/__init__.py | 40 +++++++++ python/pyspark/ml/classification.py | 49 +++++------ python/pyspark/ml/feature.py | 85 +++++++++++++++++++ python/pyspark/ml/param.py | 6 -- python/pyspark/ml/test.py | 15 ++++ 7 files changed, 200 insertions(+), 35 deletions(-) create mode 100644 examples/src/main/python/ml/simple_text_classification_pipeline.py create mode 100644 python/pyspark/ml/feature.py create mode 100644 python/pyspark/ml/test.py diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py new file mode 100644 index 0000000000000..f5558eaebe005 --- /dev/null +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -0,0 +1,33 @@ +from pyspark import SparkContext +from pyspark.sql import SQLContext, Row +from pyspark.ml import Pipeline +from pyspark.ml.feature import HashingTF, Tokenizer +from pyspark.ml.classification import LogisticRegression + +if __name__ == "__main__": + sc = SparkContext(appName="SimpleTextClassificationPipeline") + sqlCtx = SQLContext(sc) + training = sqlCtx.inferSchema( + sc.parallelize([(0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), (3L, "hadoop mapreduce", 0.0)]) \ + .map(lambda x: Row(id=x[0], text=x[1], label=x[2]))) + + tokenizer = Tokenizer() \ + .setInputCol("text") \ + .setOutputCol("words") + hashingTF = HashingTF() \ + .setInputCol(tokenizer.getOutputCol()) \ + .setOutputCol("features") + lr = LogisticRegression() \ + .setMaxIter(10) \ + .setRegParam(0.01) + pipeline = Pipeline() \ + .setStages([tokenizer, hashingTF, lr]) + + model = pipeline.fit(training) + + test = sqlCtx.inferSchema( + sc.parallelize([(4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), (7L, "apache hadoop")]) \ + .map(lambda x: Row(id=x[0], text=x[1]))) + + for row in model.transform(test).collect(): + print row diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 04f9cfb1bfc2f..33f7a3900a98e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -164,6 +164,13 @@ trait Params extends Identifiable with Serializable { this } + /** + * Sets a parameter (by name) in the embedded param map. + */ + private[ml] def set(param: String, value: Any): this.type = { + set(getParam(param), value) + } + /** * Gets the value of a parameter in the embedded param map. */ diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 622acb96782e8..f5d3557a354d4 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -1,6 +1,9 @@ import inspect from pyspark import SparkContext +from pyspark.ml.param import Param + +__all__ = ["Pipeline"] # An implementation of PEP3102 for Python 2. _keyword_only_secret = 70861589 @@ -20,3 +23,40 @@ def _assert_keyword_only_args(): def _jvm(): return SparkContext._jvm + +class Pipeline(object): + + def __init__(self): + self.stages = Param(self, "stages", "pipeline stages") + self.paramMap = {} + + def setStages(self, value): + self.paramMap[self.stages] = value + return self + + def getStages(self): + if self.stages in self.paramMap: + return self.paramMap[self.stages] + + def fit(self, dataset): + transformers = [] + for stage in self.getStages(): + if hasattr(stage, "transform"): + transformers.append(stage) + dataset = stage.transform(dataset) + elif hasattr(stage, "fit"): + model = stage.fit(dataset) + transformers.append(model) + dataset = model.transform(dataset) + return PipelineModel(transformers) + + +class PipelineModel(object): + + def __init__(self, transformers): + self.transformers = transformers + + def transform(self, dataset): + for t in self.transformers: + dataset = t.transform(dataset) + return dataset diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 9fe81e864855f..13bbe69628df1 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1,5 +1,5 @@ from pyspark.sql import SchemaRDD -from pyspark.ml import _keyword_only_secret, _assert_keyword_only_args, _jvm +from pyspark.ml import _jvm from pyspark.ml.param import Param @@ -8,45 +8,39 @@ class LogisticRegression(object): Logistic regression. """ - _java_class = "org.apache.spark.ml.classification.LogisticRegression" + # _java_class = "org.apache.spark.ml.classification.LogisticRegression" def __init__(self): self._java_obj = _jvm().org.apache.spark.ml.classification.LogisticRegression() - self.paramMap = {} self.maxIter = Param(self, "maxIter", "max number of iterations", 100) self.regParam = Param(self, "regParam", "regularization constant", 0.1) + self.featuresCol = Param(self, "featuresCol", "features column name", "features") - def set(self, _keyword_only=_keyword_only_secret, - maxIter=None, regParam=None): - _assert_keyword_only_args() - if maxIter is not None: - self.paramMap[self.maxIter] = maxIter - if regParam is not None: - self.paramMap[self.regParam] = regParam - return self - - # cannot chained def setMaxIter(self, value): - self.paramMap[self.maxIter] = value + self._java_obj.setMaxIter(value) return self + def getMaxIter(self): + return self._java_obj.getMaxIter() + def setRegParam(self, value): - self.paramMap[self.regParam] = value + self._java_obj.setRegParam(value) return self - def getMaxIter(self): - if self.maxIter in self.paramMap: - return self.paramMap[self.maxIter] - else: - return self.maxIter.defaultValue - def getRegParam(self): - if self.regParam in self.paramMap: - return self.paramMap[self.regParam] - else: - return self.regParam.defaultValue + return self._java_obj.getRegParam() + + def setFeaturesCol(self, value): + self._java_obj.setFeaturesCol(value) + return self - def fit(self, dataset): + def getFeaturesCol(self): + return self._java_obj.getFeaturesCol() + + def fit(self, dataset, params=None): + """ + Fits a dataset with optional parameters. + """ java_model = self._java_obj.fit(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()) return LogisticRegressionModel(java_model) @@ -62,6 +56,3 @@ def __init__(self, _java_model): def transform(self, dataset): return SchemaRDD(self._java_model.transform(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()), dataset.sql_ctx) -lr = LogisticRegression() - -lr.set(maxIter=10, regParam=0.1) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py new file mode 100644 index 0000000000000..23923204b60a9 --- /dev/null +++ b/python/pyspark/ml/feature.py @@ -0,0 +1,85 @@ +from pyspark.sql import SchemaRDD, ArrayType, StringType +from pyspark.ml import _jvm +from pyspark.ml.param import Param + + +class Tokenizer(object): + + def __init__(self): + self.inputCol = Param(self, "inputCol", "input column name", None) + self.outputCol = Param(self, "outputCol", "output column name", None) + self.paramMap = {} + + def setInputCol(self, value): + self.paramMap[self.inputCol] = value + return self + + def getInputCol(self): + if self.inputCol in self.paramMap: + return self.paramMap[self.inputCol] + + def setOutputCol(self, value): + self.paramMap[self.outputCol] = value + return self + + def getOutputCol(self): + if self.outputCol in self.paramMap: + return self.paramMap[self.outputCol] + + def transform(self, dataset, params={}): + sqlCtx = dataset.sql_ctx + if isinstance(params, dict): + paramMap = self.paramMap.copy() + paramMap.update(params) + inputCol = paramMap[self.inputCol] + outputCol = paramMap[self.outputCol] + # TODO: make names unique + sqlCtx.registerFunction("tokenize", lambda text: text.split(), + ArrayType(StringType(), False)) + dataset.registerTempTable("dataset") + return sqlCtx.sql("SELECT *, tokenize(%s) AS %s FROM dataset" % (inputCol, outputCol)) + elif isinstance(params, list): + return [self.transform(dataset, paramMap) for paramMap in params] + else: + raise ValueError("The input params must be either a dict or a list.") + + +class HashingTF(object): + + def __init__(self): + self._java_obj = _jvm().org.apache.spark.ml.feature.HashingTF() + self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) + self.inputCol = Param(self, "inputCol", "input column name") + self.outputCol = Param(self, "outputCol", "output column name") + + def setNumFeatures(self, value): + self._java_obj.setNumFeatures(value) + return self + + def getNumFeatures(self): + return self._java_obj.getNumFeatures() + + def setInputCol(self, value): + self._java_obj.setInputCol(value) + return self + + def getInputCol(self): + return self._java_obj.getInputCol() + + def setOutputCol(self, value): + self._java_obj.setOutputCol(value) + return self + + def getOutputCol(self): + return self._java_obj.getOutputCol() + + def transform(self, dataset, paramMap={}): + if isinstance(paramMap, dict): + javaParamMap = _jvm().org.apache.spark.ml.param.ParamMap() + for k, v in paramMap.items(): + param = self._java_obj.getParam(k.name) + javaParamMap.put(param, v) + return SchemaRDD(self._java_obj.transform(dataset._jschema_rdd, javaParamMap), + dataset.sql_ctx) + else: + raise ValueError("paramMap must be a dict.") diff --git a/python/pyspark/ml/param.py b/python/pyspark/ml/param.py index be85675233cde..181a158cb94c8 100644 --- a/python/pyspark/ml/param.py +++ b/python/pyspark/ml/param.py @@ -14,9 +14,3 @@ def __str__(self): def __repr_(self): return self.parent + "_" + self.name - - -class Params(object): - """ - Components that take parameters. - """ diff --git a/python/pyspark/ml/test.py b/python/pyspark/ml/test.py new file mode 100644 index 0000000000000..aad7483488ad7 --- /dev/null +++ b/python/pyspark/ml/test.py @@ -0,0 +1,15 @@ +import subprocess + +def funcA(dataset, **kwargs): + """ + funcA + :param dataset: + :param kwargs: + + :return: + """ + pass + + +dataset = [] +funcA(dataset, ) From a3015cf6d53bc4e71aef96461f75776945d4fcd7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 31 Dec 2014 15:39:38 -0800 Subject: [PATCH 03/28] add Estimator and Transformer --- python/pyspark/ml/__init__.py | 14 +++++++++++++- python/pyspark/ml/classification.py | 6 +++--- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index f5d3557a354d4..90cf56b97e093 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -3,7 +3,7 @@ from pyspark import SparkContext from pyspark.ml.param import Param -__all__ = ["Pipeline"] +__all__ = ["Pipeline", "Transformer", "Estimator"] # An implementation of PEP3102 for Python 2. _keyword_only_secret = 70861589 @@ -60,3 +60,15 @@ def transform(self, dataset): for t in self.transformers: dataset = t.transform(dataset) return dataset + + +class Estimator(object): + + def fit(self, dataset, params={}): + raise NotImplementedError() + + +class Transformer(object): + + def transform(self, dataset, paramMap={}): + raise NotImplementedError() diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 13bbe69628df1..957d93d45cbf7 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1,9 +1,9 @@ from pyspark.sql import SchemaRDD -from pyspark.ml import _jvm +from pyspark.ml import Estimator, Transformer, _jvm from pyspark.ml.param import Param -class LogisticRegression(object): +class LogisticRegression(Estimator): """ Logistic regression. """ @@ -45,7 +45,7 @@ def fit(self, dataset, params=None): return LogisticRegressionModel(java_model) -class LogisticRegressionModel(object): +class LogisticRegressionModel(Transformer): """ Model fitted by LogisticRegression. """ From dadd84ee3d5f70b6a9b2af286cea9cac2057a764 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 19 Jan 2015 10:05:15 -0800 Subject: [PATCH 04/28] add base classes and docs --- python/docs/pyspark.ml.rst | 13 +++ python/pyspark/ml/__init__.py | 144 +++++++++++++++++++++------- python/pyspark/ml/classification.py | 25 ++++- python/pyspark/ml/feature.py | 17 ++++ python/pyspark/ml/param.py | 50 +++++++++- python/pyspark/ml/test.py | 15 --- python/pyspark/ml/util.py | 27 ++++++ 7 files changed, 240 insertions(+), 51 deletions(-) create mode 100644 python/docs/pyspark.ml.rst delete mode 100644 python/pyspark/ml/test.py create mode 100644 python/pyspark/ml/util.py diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst new file mode 100644 index 0000000000000..9015a3c15edb2 --- /dev/null +++ b/python/docs/pyspark.ml.rst @@ -0,0 +1,13 @@ +pyspark.ml package +===================== + +Submodules +---------- + +pyspark.ml module +------------------------- + +.. automodule:: pyspark.ml + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 90cf56b97e093..b6606c76063db 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -1,57 +1,149 @@ -import inspect +# +# 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. +# + +from abc import ABCMeta, abstractmethod from pyspark import SparkContext -from pyspark.ml.param import Param +from pyspark.sql import inherit_doc +from pyspark.ml.param import Param, Params +from pyspark.ml.util import Identifiable __all__ = ["Pipeline", "Transformer", "Estimator"] -# An implementation of PEP3102 for Python 2. -_keyword_only_secret = 70861589 + +def _jvm(): + return SparkContext._jvm -def _assert_keyword_only_args(): +@inherit_doc +class PipelineStage(Params): """ - Checks whether the _keyword_only trick is applied and validates input arguments. + A stage in a pipeline, either an :py:class:`Estimator` or a + :py:class:`Transformer`. """ - # Get the frame of the function that calls this function. - frame = inspect.currentframe().f_back - info = inspect.getargvalues(frame) - if "_keyword_only" not in info.args: - raise ValueError("Function does not have argument _keyword_only.") - if info.locals["_keyword_only"] != _keyword_only_secret: - raise ValueError("Must use keyword arguments instead of positional ones.") -def _jvm(): - return SparkContext._jvm + def __init__(self): + super.__init__(self) + + +@inherit_doc +class Estimator(PipelineStage): + """ + Abstract class for estimators that fit models to data. + """ + + __metaclass__ = ABCMeta + + def __init__(self): + super.__init__(self) + + @abstractmethod + def fit(self, dataset, params={}): + """ + Fits a model to the input dataset with optional parameters. + + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: an optional param map that overwrites embedded + params + :returns: fitted model + """ + raise NotImplementedError() + + +@inherit_doc +class Transformer(PipelineStage): + """ + Abstract class for transformers that transform one dataset into + another. + """ + + __metaclass__ = ABCMeta -class Pipeline(object): + @abstractmethod + def transform(self, dataset, params={}): + """ + Transforms the input dataset with optional parameters. + + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: an optional param map that overwrites embedded + params + :returns: transformed dataset + """ + raise NotImplementedError() + + +@inherit_doc +class Pipeline(Estimator): + """ + A simple pipeline, which acts as an estimator. A Pipeline consists + of a sequence of stages, each of which is either an + :py:class:`Estimator` or a :py:class:`Transformer`. When + :py:meth:`Pipeline.fit` is called, the stages are executed in + order. If a stage is an :py:class:`Estimator`, its + :py:meth:`Estimator.fit` method will be called on the input + dataset to fit a model. Then the model, which is a transformer, + will be used to transform the dataset as the input to the next + stage. If a stage is a :py:class:`Transformer`, its + :py:meth:`Transformer.transform` method will be called to produce + the dataset for the next stage. The fitted model from a + :py:class:`Pipeline` is an :py:class:`PipelineModel`, which + consists of fitted models and transformers, corresponding to the + pipeline stages. If there are no stages, the pipeline acts as an + identity transformer. + """ def __init__(self): + super.__init__(self) + #: Param for pipeline stages. self.stages = Param(self, "stages", "pipeline stages") - self.paramMap = {} def setStages(self, value): + """ + Set pipeline stages. + :param value: a list of transformers or estimators + :return: the pipeline instance + """ self.paramMap[self.stages] = value return self def getStages(self): + """ + Get pipeline stages. + """ if self.stages in self.paramMap: return self.paramMap[self.stages] def fit(self, dataset): transformers = [] for stage in self.getStages(): - if hasattr(stage, "transform"): + if isinstance(stage, Transformer): transformers.append(stage) dataset = stage.transform(dataset) - elif hasattr(stage, "fit"): + elif isinstance(stage, Estimator): model = stage.fit(dataset) transformers.append(model) dataset = model.transform(dataset) return PipelineModel(transformers) -class PipelineModel(object): +@inherit_doc +class PipelineModel(Transformer): def __init__(self, transformers): self.transformers = transformers @@ -60,15 +152,3 @@ def transform(self, dataset): for t in self.transformers: dataset = t.transform(dataset) return dataset - - -class Estimator(object): - - def fit(self, dataset, params={}): - raise NotImplementedError() - - -class Transformer(object): - - def transform(self, dataset, paramMap={}): - raise NotImplementedError() diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 957d93d45cbf7..2c9aaad03cedf 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1,3 +1,20 @@ +# +# 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. +# + from pyspark.sql import SchemaRDD from pyspark.ml import Estimator, Transformer, _jvm from pyspark.ml.param import Param @@ -41,7 +58,8 @@ def fit(self, dataset, params=None): """ Fits a dataset with optional parameters. """ - java_model = self._java_obj.fit(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()) + java_model = self._java_obj.fit(dataset._jschema_rdd, + _jvm().org.apache.spark.ml.param.ParamMap()) return LogisticRegressionModel(java_model) @@ -54,5 +72,6 @@ def __init__(self, _java_model): self._java_model = _java_model def transform(self, dataset): - return SchemaRDD(self._java_model.transform(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()), dataset.sql_ctx) - + return SchemaRDD(self._java_model.transform( + dataset._jschema_rdd, + _jvm().org.apache.spark.ml.param.ParamMap()), dataset.sql_ctx) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 23923204b60a9..ce45105ba2b28 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1,3 +1,20 @@ +# +# 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. +# + from pyspark.sql import SchemaRDD, ArrayType, StringType from pyspark.ml import _jvm from pyspark.ml.param import Param diff --git a/python/pyspark/ml/param.py b/python/pyspark/ml/param.py index 181a158cb94c8..427a70cc11d5c 100644 --- a/python/pyspark/ml/param.py +++ b/python/pyspark/ml/param.py @@ -1,3 +1,28 @@ +# +# 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. +# + +from abc import ABCMeta, abstractmethod + +from pyspark.ml.util import Identifiable + + +__all__ = ["Param"] + + class Param(object): """ A param with self-contained documentation and optionally default value. @@ -12,5 +37,28 @@ def __init__(self, parent, name, doc, defaultValue=None): def __str__(self): return self.parent + "_" + self.name - def __repr_(self): + def __repr__(self): return self.parent + "_" + self.name + + +class Params(Identifiable): + """ + Components that take parameters. This also provides an internal + param map to store parameter values attached to the instance. + """ + + __metaclass__ = ABCMeta + + def __init__(self): + super.__init__(self) + #: Internal param map. + self.paramMap = {} + + @abstractmethod + def params(self): + """ + Returns all params. The default implementation uses + :py:func:`dir` to get all attributes of type + :py:class:`Param`. + """ + return [attr for attr in dir(self) if isinstance(attr, Param)] diff --git a/python/pyspark/ml/test.py b/python/pyspark/ml/test.py deleted file mode 100644 index aad7483488ad7..0000000000000 --- a/python/pyspark/ml/test.py +++ /dev/null @@ -1,15 +0,0 @@ -import subprocess - -def funcA(dataset, **kwargs): - """ - funcA - :param dataset: - :param kwargs: - - :return: - """ - pass - - -dataset = [] -funcA(dataset, ) diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py new file mode 100644 index 0000000000000..c6561a13a5d9d --- /dev/null +++ b/python/pyspark/ml/util.py @@ -0,0 +1,27 @@ +# +# 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. +# + + +class Identifiable(object): + """ + Object with a unique ID. + """ + + def __init__(self): + #: A unique id for the object. The default implementation + #: concatenates the class name, "-", and 8 random hex chars. + self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] From c18dca1bf86d7ec9602190d35fd13b607e829f7b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 19 Jan 2015 12:08:18 -0800 Subject: [PATCH 05/28] make the example working --- .../ml/simple_text_classification_pipeline.py | 27 +++++++++++++++++-- python/pyspark/ml/__init__.py | 15 ++++++++--- python/pyspark/ml/feature.py | 9 ++++--- python/pyspark/ml/param.py | 5 ++-- python/pyspark/ml/util.py | 2 ++ 5 files changed, 45 insertions(+), 13 deletions(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index f5558eaebe005..d303d4d97de79 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -1,3 +1,20 @@ +# +# 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. +# + from pyspark import SparkContext from pyspark.sql import SQLContext, Row from pyspark.ml import Pipeline @@ -8,7 +25,10 @@ sc = SparkContext(appName="SimpleTextClassificationPipeline") sqlCtx = SQLContext(sc) training = sqlCtx.inferSchema( - sc.parallelize([(0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), (3L, "hadoop mapreduce", 0.0)]) \ + sc.parallelize([(0L, "a b c d e spark", 1.0), + (1L, "b d", 0.0), + (2L, "spark f g h", 1.0), + (3L, "hadoop mapreduce", 0.0)]) \ .map(lambda x: Row(id=x[0], text=x[1], label=x[2]))) tokenizer = Tokenizer() \ @@ -26,7 +46,10 @@ model = pipeline.fit(training) test = sqlCtx.inferSchema( - sc.parallelize([(4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), (7L, "apache hadoop")]) \ + sc.parallelize([(4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop")]) \ .map(lambda x: Row(id=x[0], text=x[1]))) for row in model.transform(test).collect(): diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index b6606c76063db..df39515d81433 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -18,7 +18,7 @@ from abc import ABCMeta, abstractmethod from pyspark import SparkContext -from pyspark.sql import inherit_doc +from pyspark.sql import inherit_doc # TODO: move inherit_doc to Spark Core from pyspark.ml.param import Param, Params from pyspark.ml.util import Identifiable @@ -37,7 +37,7 @@ class PipelineStage(Params): """ def __init__(self): - super.__init__(self) + super(PipelineStage, self).__init__() @inherit_doc @@ -49,7 +49,7 @@ class Estimator(PipelineStage): __metaclass__ = ABCMeta def __init__(self): - super.__init__(self) + super(Estimator, self).__init__() @abstractmethod def fit(self, dataset, params={}): @@ -74,6 +74,9 @@ class Transformer(PipelineStage): __metaclass__ = ABCMeta + def __init__(self): + super(Transformer, self).__init__() + @abstractmethod def transform(self, dataset, params={}): """ @@ -109,7 +112,7 @@ class Pipeline(Estimator): """ def __init__(self): - super.__init__(self) + super(Pipeline, self).__init__() #: Param for pipeline stages. self.stages = Param(self, "stages", "pipeline stages") @@ -139,6 +142,9 @@ def fit(self, dataset): model = stage.fit(dataset) transformers.append(model) dataset = model.transform(dataset) + else: + raise ValueError( + "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) return PipelineModel(transformers) @@ -146,6 +152,7 @@ def fit(self, dataset): class PipelineModel(Transformer): def __init__(self, transformers): + super(PipelineModel, self).__init__() self.transformers = transformers def transform(self, dataset): diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index ce45105ba2b28..35f32f513f7a5 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -16,13 +16,13 @@ # from pyspark.sql import SchemaRDD, ArrayType, StringType -from pyspark.ml import _jvm +from pyspark.ml import Transformer, _jvm from pyspark.ml.param import Param - -class Tokenizer(object): +class Tokenizer(Transformer): def __init__(self): + super(Tokenizer, self).__init__() self.inputCol = Param(self, "inputCol", "input column name", None) self.outputCol = Param(self, "outputCol", "output column name", None) self.paramMap = {} @@ -61,9 +61,10 @@ def transform(self, dataset, params={}): raise ValueError("The input params must be either a dict or a list.") -class HashingTF(object): +class HashingTF(Transformer): def __init__(self): + super(HashingTF, self).__init__() self._java_obj = _jvm().org.apache.spark.ml.feature.HashingTF() self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) self.inputCol = Param(self, "inputCol", "input column name") diff --git a/python/pyspark/ml/param.py b/python/pyspark/ml/param.py index 427a70cc11d5c..ffe58a6ee69d7 100644 --- a/python/pyspark/ml/param.py +++ b/python/pyspark/ml/param.py @@ -15,7 +15,7 @@ # limitations under the License. # -from abc import ABCMeta, abstractmethod +from abc import ABCMeta from pyspark.ml.util import Identifiable @@ -50,11 +50,10 @@ class Params(Identifiable): __metaclass__ = ABCMeta def __init__(self): - super.__init__(self) + super(Params, self).__init__() #: Internal param map. self.paramMap = {} - @abstractmethod def params(self): """ Returns all params. The default implementation uses diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index c6561a13a5d9d..801a5eeaa3249 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -15,6 +15,8 @@ # limitations under the License. # +import uuid + class Identifiable(object): """ From d9ea77cd084394bf7e38409b8250900cb237c003 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 19 Jan 2015 13:32:32 -0800 Subject: [PATCH 06/28] update doc --- python/docs/pyspark.ml.rst | 26 +++++++++++++++++++++++++- python/pyspark/ml/__init__.py | 2 +- python/pyspark/ml/classification.py | 3 ++- python/pyspark/ml/feature.py | 4 +++- 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index 9015a3c15edb2..f20453e624296 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -5,9 +5,33 @@ Submodules ---------- pyspark.ml module -------------------------- +----------------- .. automodule:: pyspark.ml :members: :undoc-members: :show-inheritance: + +pyspark.ml.param module +----------------------- + +.. automodule:: pyspark.ml.param + :members: + :undoc-members: + :show-inheritance: + +pyspark.ml.feature module +------------------------- + +.. automodule:: pyspark.ml.feature + :members: + :undoc-members: + :show-inheritance: + +pyspark.ml.classification module +-------------------------------- + +.. automodule:: pyspark.ml.classification + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index df39515d81433..83351dc523ce1 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -22,7 +22,7 @@ from pyspark.ml.param import Param, Params from pyspark.ml.util import Identifiable -__all__ = ["Pipeline", "Transformer", "Estimator"] +__all__ = ["Pipeline", "Transformer", "Estimator", "param", "feature", "classification"] def _jvm(): diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 2c9aaad03cedf..f3cea0958c897 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -15,11 +15,12 @@ # limitations under the License. # -from pyspark.sql import SchemaRDD +from pyspark.sql import SchemaRDD, inherit_doc from pyspark.ml import Estimator, Transformer, _jvm from pyspark.ml.param import Param +@inherit_doc class LogisticRegression(Estimator): """ Logistic regression. diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 35f32f513f7a5..9e4b6574574a4 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,10 +15,11 @@ # limitations under the License. # -from pyspark.sql import SchemaRDD, ArrayType, StringType +from pyspark.sql import SchemaRDD, ArrayType, StringType, inherit_doc from pyspark.ml import Transformer, _jvm from pyspark.ml.param import Param +@inherit_doc class Tokenizer(Transformer): def __init__(self): @@ -61,6 +62,7 @@ def transform(self, dataset, params={}): raise ValueError("The input params must be either a dict or a list.") +@inherit_doc class HashingTF(Transformer): def __init__(self): From 17ecfb9f681e136f4e9be9ecdf5f468056dad6d4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 21 Jan 2015 15:31:00 -0800 Subject: [PATCH 07/28] code gen for shared params --- python/docs/pyspark.ml.rst | 1 + python/pyspark/ml/__init__.py | 14 ++ python/pyspark/ml/classification.py | 31 +--- .../ml/{param.py => param/__init__.py} | 18 ++- python/pyspark/ml/param/_gen_shared_params.py | 80 ++++++++++ python/pyspark/ml/param/shared.py | 146 ++++++++++++++++++ python/pyspark/ml/util.py | 6 + 7 files changed, 262 insertions(+), 34 deletions(-) rename python/pyspark/ml/{param.py => param/__init__.py} (78%) create mode 100644 python/pyspark/ml/param/_gen_shared_params.py create mode 100644 python/pyspark/ml/param/shared.py diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index f20453e624296..f4c1a6406e020 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -34,4 +34,5 @@ pyspark.ml.classification module .. automodule:: pyspark.ml.classification :members: :undoc-members: + :inherited-members: :show-inheritance: diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 83351dc523ce1..4666ce7bc2499 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -29,6 +29,20 @@ def _jvm(): return SparkContext._jvm +def _inherit_doc(cls): + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls + + @inherit_doc class PipelineStage(Params): """ diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index f3cea0958c897..fd1fb906ca5c1 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -17,11 +17,13 @@ from pyspark.sql import SchemaRDD, inherit_doc from pyspark.ml import Estimator, Transformer, _jvm -from pyspark.ml.param import Param +from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ + HasRegParam @inherit_doc -class LogisticRegression(Estimator): +class LogisticRegression(Estimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, + HasRegParam): """ Logistic regression. """ @@ -29,31 +31,8 @@ class LogisticRegression(Estimator): # _java_class = "org.apache.spark.ml.classification.LogisticRegression" def __init__(self): + super(LogisticRegression, self).__init__() self._java_obj = _jvm().org.apache.spark.ml.classification.LogisticRegression() - self.maxIter = Param(self, "maxIter", "max number of iterations", 100) - self.regParam = Param(self, "regParam", "regularization constant", 0.1) - self.featuresCol = Param(self, "featuresCol", "features column name", "features") - - def setMaxIter(self, value): - self._java_obj.setMaxIter(value) - return self - - def getMaxIter(self): - return self._java_obj.getMaxIter() - - def setRegParam(self, value): - self._java_obj.setRegParam(value) - return self - - def getRegParam(self): - return self._java_obj.getRegParam() - - def setFeaturesCol(self, value): - self._java_obj.setFeaturesCol(value) - return self - - def getFeaturesCol(self): - return self._java_obj.getFeaturesCol() def fit(self, dataset, params=None): """ diff --git a/python/pyspark/ml/param.py b/python/pyspark/ml/param/__init__.py similarity index 78% rename from python/pyspark/ml/param.py rename to python/pyspark/ml/param/__init__.py index ffe58a6ee69d7..89e5d732f7586 100644 --- a/python/pyspark/ml/param.py +++ b/python/pyspark/ml/param/__init__.py @@ -19,7 +19,6 @@ from pyspark.ml.util import Identifiable - __all__ = ["Param"] @@ -29,16 +28,18 @@ class Param(object): """ def __init__(self, parent, name, doc, defaultValue=None): + if not isinstance(parent, Identifiable): + raise ValueError("Parent must be identifiable but got type %s." % type(parent).__name__) self.parent = parent - self.name = name - self.doc = doc + self.name = str(name) + self.doc = str(doc) self.defaultValue = defaultValue def __str__(self): - return self.parent + "_" + self.name + return str(self.parent) + "_" + self.name def __repr__(self): - return self.parent + "_" + self.name + return str(self.parent) + "_" + self.name class Params(Identifiable): @@ -49,10 +50,11 @@ class Params(Identifiable): __metaclass__ = ABCMeta + #: Internal param map. + paramMap = {} + def __init__(self): super(Params, self).__init__() - #: Internal param map. - self.paramMap = {} def params(self): """ @@ -60,4 +62,4 @@ def params(self): :py:func:`dir` to get all attributes of type :py:class:`Param`. """ - return [attr for attr in dir(self) if isinstance(attr, Param)] + return filter(lambda x: isinstance(x, Param), map(lambda x: getattr(self, x), dir(self))) diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_gen_shared_params.py new file mode 100644 index 0000000000000..8c3aa7eba9483 --- /dev/null +++ b/python/pyspark/ml/param/_gen_shared_params.py @@ -0,0 +1,80 @@ +# +# 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. +# + +header = """# +# 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. +#""" + + +def _gen_param_code(name, doc, defaultValue): + """ + Generates Python code for a shared param class. + + :param name: param name + :param doc: param doc + :param defaultValue: string representation of the param + :return: code string + """ + upperCamelName = name[0].upper() + name[1:] + return """class Has%s(Params): + + def __init__(self): + super(Has%s, self).__init__() + #: %s + self.%s = Param(self, "%s", "%s", %s) + + def set%s(self, value): + self.paramMap[self.%s] = value + return self + + def get%s(self, value): + if self.%s in self.paramMap: + return self.paramMap[self.%s] + else: + return self.defaultValue""" % ( + upperCamelName, upperCamelName, doc, name, name, doc, defaultValue, upperCamelName, name, + upperCamelName, name, name) + +if __name__ == "__main__": + print header + print "\n# DO NOT MODIFY. The code is generated by _gen_shared_params.py.\n" + print "from pyspark.ml.param import Param, Params\n\n" + shared = [ + ("maxIter", "max number of iterations", "100"), + ("regParam", "regularization constant", "0.1"), + ("featuresCol", "features column name", "'features'"), + ("labelCol", "label column name", "'label'"), + ("predictionCol", "prediction column name", "'prediction'"), + ("inputCol", "input column name", "'input'"), + ("outputCol", "output column name", "'output'")] + code = [] + for name, doc, defaultValue in shared: + code.append(_gen_param_code(name, doc, defaultValue)) + print "\n\n\n".join(code) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py new file mode 100644 index 0000000000000..88afb5481f7b8 --- /dev/null +++ b/python/pyspark/ml/param/shared.py @@ -0,0 +1,146 @@ +# +# 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. +# + +# DO NOT MODIFY. The code is generated by _gen_shared_params.py. + +from pyspark.ml.param import Param, Params + + +class HasMaxIter(Params): + + def __init__(self): + super(HasMaxIter, self).__init__() + #: max number of iterations + self.maxIter = Param(self, "maxIter", "max number of iterations", 100) + + def setMaxIter(self, value): + self.paramMap[self.maxIter] = value + return self + + def getMaxIter(self, value): + if self.maxIter in self.paramMap: + return self.paramMap[self.maxIter] + else: + return self.defaultValue + + +class HasRegParam(Params): + + def __init__(self): + super(HasRegParam, self).__init__() + #: regularization constant + self.regParam = Param(self, "regParam", "regularization constant", 0.1) + + def setRegParam(self, value): + self.paramMap[self.regParam] = value + return self + + def getRegParam(self, value): + if self.regParam in self.paramMap: + return self.paramMap[self.regParam] + else: + return self.defaultValue + + +class HasFeaturesCol(Params): + + def __init__(self): + super(HasFeaturesCol, self).__init__() + #: features column name + self.featuresCol = Param(self, "featuresCol", "features column name", 'features') + + def setFeaturesCol(self, value): + self.paramMap[self.featuresCol] = value + return self + + def getFeaturesCol(self, value): + if self.featuresCol in self.paramMap: + return self.paramMap[self.featuresCol] + else: + return self.defaultValue + + +class HasLabelCol(Params): + + def __init__(self): + super(HasLabelCol, self).__init__() + #: label column name + self.labelCol = Param(self, "labelCol", "label column name", 'label') + + def setLabelCol(self, value): + self.paramMap[self.labelCol] = value + return self + + def getLabelCol(self, value): + if self.labelCol in self.paramMap: + return self.paramMap[self.labelCol] + else: + return self.defaultValue + + +class HasPredictionCol(Params): + + def __init__(self): + super(HasPredictionCol, self).__init__() + #: prediction column name + self.predictionCol = Param(self, "predictionCol", "prediction column name", 'prediction') + + def setPredictionCol(self, value): + self.paramMap[self.predictionCol] = value + return self + + def getPredictionCol(self, value): + if self.predictionCol in self.paramMap: + return self.paramMap[self.predictionCol] + else: + return self.defaultValue + + +class HasInputCol(Params): + + def __init__(self): + super(HasInputCol, self).__init__() + #: input column name + self.inputCol = Param(self, "inputCol", "input column name", 'input') + + def setInputCol(self, value): + self.paramMap[self.inputCol] = value + return self + + def getInputCol(self, value): + if self.inputCol in self.paramMap: + return self.paramMap[self.inputCol] + else: + return self.defaultValue + + +class HasOutputCol(Params): + + def __init__(self): + super(HasOutputCol, self).__init__() + #: output column name + self.outputCol = Param(self, "outputCol", "output column name", 'output') + + def setOutputCol(self, value): + self.paramMap[self.outputCol] = value + return self + + def getOutputCol(self, value): + if self.outputCol in self.paramMap: + return self.paramMap[self.outputCol] + else: + return self.defaultValue diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 801a5eeaa3249..5d74088b0b13e 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -27,3 +27,9 @@ def __init__(self): #: A unique id for the object. The default implementation #: concatenates the class name, "-", and 8 random hex chars. self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + + def __str__(self): + return self.uid + + def __repr__(self): + return str(self) From d0c5bb83ebe0055b22f2cd73c02f6281562177b9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 21 Jan 2015 16:51:36 -0800 Subject: [PATCH 08/28] a working copy --- .../ml/simple_text_classification_pipeline.py | 1 + python/pyspark/ml/__init__.py | 75 ++++++++++++++-- python/pyspark/ml/classification.py | 28 +++--- python/pyspark/ml/feature.py | 86 ++++--------------- python/pyspark/ml/param/__init__.py | 11 +++ python/pyspark/ml/param/_gen_shared_params.py | 6 +- python/pyspark/ml/param/shared.py | 28 +++--- 7 files changed, 127 insertions(+), 108 deletions(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index d303d4d97de79..eb80f44f4fa97 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -21,6 +21,7 @@ from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.ml.classification import LogisticRegression + if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") sqlCtx = SQLContext(sc) diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 4666ce7bc2499..1cf9d3065f3d1 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -15,10 +15,10 @@ # limitations under the License. # -from abc import ABCMeta, abstractmethod +from abc import ABCMeta, abstractmethod, abstractproperty from pyspark import SparkContext -from pyspark.sql import inherit_doc # TODO: move inherit_doc to Spark Core +from pyspark.sql import SchemaRDD, inherit_doc # TODO: move inherit_doc to Spark Core from pyspark.ml.param import Param, Params from pyspark.ml.util import Identifiable @@ -146,16 +146,17 @@ def getStages(self): if self.stages in self.paramMap: return self.paramMap[self.stages] - def fit(self, dataset): + def fit(self, dataset, params={}): + map = self._merge_params(params) transformers = [] for stage in self.getStages(): if isinstance(stage, Transformer): transformers.append(stage) - dataset = stage.transform(dataset) + dataset = stage.transform(dataset, map) elif isinstance(stage, Estimator): - model = stage.fit(dataset) + model = stage.fit(dataset, map) transformers.append(model) - dataset = model.transform(dataset) + dataset = model.transform(dataset, map) else: raise ValueError( "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) @@ -169,7 +170,65 @@ def __init__(self, transformers): super(PipelineModel, self).__init__() self.transformers = transformers - def transform(self, dataset): + def transform(self, dataset, params={}): + map = self._merge_params(params) for t in self.transformers: - dataset = t.transform(dataset) + dataset = t.transform(dataset, map) return dataset + + +@inherit_doc +class JavaWrapper(object): + + __metaclass__ = ABCMeta + + def __init__(self): + super(JavaWrapper, self).__init__() + + @abstractproperty + def _java_class(self): + raise NotImplementedError + + def _create_java_obj(self): + java_obj = _jvm() + for name in self._java_class.split("."): + java_obj = getattr(java_obj, name) + return java_obj() + + +@inherit_doc +class JavaEstimator(Estimator, JavaWrapper): + + __metaclass__ = ABCMeta + + def __init__(self): + super(JavaEstimator, self).__init__() + + @abstractmethod + def _create_model(self, java_model): + raise NotImplementedError + + def _fit_java(self, dataset, params={}): + java_obj = self._create_java_obj() + self._transfer_params_to_java(params, java_obj) + return java_obj.fit(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()) + + def fit(self, dataset, params={}): + java_model = self._fit_java(dataset, params) + return self._create_model(java_model) + + +@inherit_doc +class JavaTransformer(Transformer, JavaWrapper): + + __metaclass__ = ABCMeta + + def __init__(self): + super(JavaTransformer, self).__init__() + + def transform(self, dataset, params={}): + java_obj = self._create_java_obj() + self._transfer_params_to_java(params, java_obj) + return SchemaRDD(java_obj.transform(dataset._jschema_rdd, + _jvm().org.apache.spark.ml.param.ParamMap()), + dataset.sql_ctx) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index fd1fb906ca5c1..3a105f1a10a3d 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -16,42 +16,40 @@ # from pyspark.sql import SchemaRDD, inherit_doc -from pyspark.ml import Estimator, Transformer, _jvm +from pyspark.ml import JavaEstimator, Transformer, _jvm from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ HasRegParam @inherit_doc -class LogisticRegression(Estimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, +class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, HasRegParam): """ Logistic regression. """ - # _java_class = "org.apache.spark.ml.classification.LogisticRegression" - def __init__(self): super(LogisticRegression, self).__init__() - self._java_obj = _jvm().org.apache.spark.ml.classification.LogisticRegression() - - def fit(self, dataset, params=None): - """ - Fits a dataset with optional parameters. - """ - java_model = self._java_obj.fit(dataset._jschema_rdd, - _jvm().org.apache.spark.ml.param.ParamMap()) + + @property + def _java_class(self): + return "org.apache.spark.ml.classification.LogisticRegression" + + def _create_model(self, java_model): return LogisticRegressionModel(java_model) +@inherit_doc class LogisticRegressionModel(Transformer): """ Model fitted by LogisticRegression. """ - def __init__(self, _java_model): - self._java_model = _java_model + def __init__(self, java_model): + self._java_model = java_model - def transform(self, dataset): + def transform(self, dataset, params={}): + # TODO: handle params here. return SchemaRDD(self._java_model.transform( dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()), dataset.sql_ctx) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9e4b6574574a4..9f3b1c7a055cb 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,91 +15,41 @@ # limitations under the License. # -from pyspark.sql import SchemaRDD, ArrayType, StringType, inherit_doc -from pyspark.ml import Transformer, _jvm +from pyspark.sql import inherit_doc +from pyspark.ml import JavaTransformer from pyspark.ml.param import Param +from pyspark.ml.param.shared import HasInputCol, HasOutputCol + @inherit_doc -class Tokenizer(Transformer): +class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): def __init__(self): super(Tokenizer, self).__init__() - self.inputCol = Param(self, "inputCol", "input column name", None) - self.outputCol = Param(self, "outputCol", "output column name", None) - self.paramMap = {} - - def setInputCol(self, value): - self.paramMap[self.inputCol] = value - return self - - def getInputCol(self): - if self.inputCol in self.paramMap: - return self.paramMap[self.inputCol] - def setOutputCol(self, value): - self.paramMap[self.outputCol] = value - return self - - def getOutputCol(self): - if self.outputCol in self.paramMap: - return self.paramMap[self.outputCol] - - def transform(self, dataset, params={}): - sqlCtx = dataset.sql_ctx - if isinstance(params, dict): - paramMap = self.paramMap.copy() - paramMap.update(params) - inputCol = paramMap[self.inputCol] - outputCol = paramMap[self.outputCol] - # TODO: make names unique - sqlCtx.registerFunction("tokenize", lambda text: text.split(), - ArrayType(StringType(), False)) - dataset.registerTempTable("dataset") - return sqlCtx.sql("SELECT *, tokenize(%s) AS %s FROM dataset" % (inputCol, outputCol)) - elif isinstance(params, list): - return [self.transform(dataset, paramMap) for paramMap in params] - else: - raise ValueError("The input params must be either a dict or a list.") + @property + def _java_class(self): + return "org.apache.spark.ml.feature.Tokenizer" @inherit_doc -class HashingTF(Transformer): +class HashingTF(JavaTransformer, HasInputCol, HasOutputCol): def __init__(self): super(HashingTF, self).__init__() - self._java_obj = _jvm().org.apache.spark.ml.feature.HashingTF() + #: param for number of features self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) - self.inputCol = Param(self, "inputCol", "input column name") - self.outputCol = Param(self, "outputCol", "output column name") + + @property + def _java_class(self): + return "org.apache.spark.ml.feature.HashingTF" def setNumFeatures(self, value): - self._java_obj.setNumFeatures(value) + self.paramMap[self.numFeatures] = value return self def getNumFeatures(self): - return self._java_obj.getNumFeatures() - - def setInputCol(self, value): - self._java_obj.setInputCol(value) - return self - - def getInputCol(self): - return self._java_obj.getInputCol() - - def setOutputCol(self, value): - self._java_obj.setOutputCol(value) - return self - - def getOutputCol(self): - return self._java_obj.getOutputCol() - - def transform(self, dataset, paramMap={}): - if isinstance(paramMap, dict): - javaParamMap = _jvm().org.apache.spark.ml.param.ParamMap() - for k, v in paramMap.items(): - param = self._java_obj.getParam(k.name) - javaParamMap.put(param, v) - return SchemaRDD(self._java_obj.transform(dataset._jschema_rdd, javaParamMap), - dataset.sql_ctx) + if self.numFeatures in self.paramMap: + return self.paramMap[self.numFeatures] else: - raise ValueError("paramMap must be a dict.") + return self.numFeatures.defaultValue diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 89e5d732f7586..f81b836a2242f 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -63,3 +63,14 @@ def params(self): :py:class:`Param`. """ return filter(lambda x: isinstance(x, Param), map(lambda x: getattr(self, x), dir(self))) + + def _merge_params(self, params): + map = self.paramMap.copy() + map.update(params) + return map + + def _transfer_params_to_java(self, params, java_obj): + map = self._merge_params(params) + for param in self.params(): + if param in map: + java_obj.set(param.name, map[param]) diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_gen_shared_params.py index 8c3aa7eba9483..f40823b906221 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_gen_shared_params.py @@ -54,13 +54,13 @@ def set%s(self, value): self.paramMap[self.%s] = value return self - def get%s(self, value): + def get%s(self): if self.%s in self.paramMap: return self.paramMap[self.%s] else: - return self.defaultValue""" % ( + return self.%s.defaultValue""" % ( upperCamelName, upperCamelName, doc, name, name, doc, defaultValue, upperCamelName, name, - upperCamelName, name, name) + upperCamelName, name, name, name) if __name__ == "__main__": print header diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 88afb5481f7b8..8680f389577b6 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -31,11 +31,11 @@ def setMaxIter(self, value): self.paramMap[self.maxIter] = value return self - def getMaxIter(self, value): + def getMaxIter(self): if self.maxIter in self.paramMap: return self.paramMap[self.maxIter] else: - return self.defaultValue + return self.maxIter.defaultValue class HasRegParam(Params): @@ -49,11 +49,11 @@ def setRegParam(self, value): self.paramMap[self.regParam] = value return self - def getRegParam(self, value): + def getRegParam(self): if self.regParam in self.paramMap: return self.paramMap[self.regParam] else: - return self.defaultValue + return self.regParam.defaultValue class HasFeaturesCol(Params): @@ -67,11 +67,11 @@ def setFeaturesCol(self, value): self.paramMap[self.featuresCol] = value return self - def getFeaturesCol(self, value): + def getFeaturesCol(self): if self.featuresCol in self.paramMap: return self.paramMap[self.featuresCol] else: - return self.defaultValue + return self.featuresCol.defaultValue class HasLabelCol(Params): @@ -85,11 +85,11 @@ def setLabelCol(self, value): self.paramMap[self.labelCol] = value return self - def getLabelCol(self, value): + def getLabelCol(self): if self.labelCol in self.paramMap: return self.paramMap[self.labelCol] else: - return self.defaultValue + return self.labelCol.defaultValue class HasPredictionCol(Params): @@ -103,11 +103,11 @@ def setPredictionCol(self, value): self.paramMap[self.predictionCol] = value return self - def getPredictionCol(self, value): + def getPredictionCol(self): if self.predictionCol in self.paramMap: return self.paramMap[self.predictionCol] else: - return self.defaultValue + return self.predictionCol.defaultValue class HasInputCol(Params): @@ -121,11 +121,11 @@ def setInputCol(self, value): self.paramMap[self.inputCol] = value return self - def getInputCol(self, value): + def getInputCol(self): if self.inputCol in self.paramMap: return self.paramMap[self.inputCol] else: - return self.defaultValue + return self.inputCol.defaultValue class HasOutputCol(Params): @@ -139,8 +139,8 @@ def setOutputCol(self, value): self.paramMap[self.outputCol] = value return self - def getOutputCol(self, value): + def getOutputCol(self): if self.outputCol in self.paramMap: return self.paramMap[self.outputCol] else: - return self.defaultValue + return self.outputCol.defaultValue From 56de5717c258b90ea5b02a6e4263f365f39628f6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 21 Jan 2015 16:53:58 -0800 Subject: [PATCH 09/28] fix style --- .../ml/simple_text_classification_pipeline.py | 18 +++++++++--------- python/pyspark/ml/__init__.py | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index eb80f44f4fa97..f457ce97a42d1 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -29,20 +29,20 @@ sc.parallelize([(0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) \ + (3L, "hadoop mapreduce", 0.0)]) .map(lambda x: Row(id=x[0], text=x[1], label=x[2]))) tokenizer = Tokenizer() \ - .setInputCol("text") \ - .setOutputCol("words") + .setInputCol("text") \ + .setOutputCol("words") hashingTF = HashingTF() \ - .setInputCol(tokenizer.getOutputCol()) \ - .setOutputCol("features") + .setInputCol(tokenizer.getOutputCol()) \ + .setOutputCol("features") lr = LogisticRegression() \ - .setMaxIter(10) \ - .setRegParam(0.01) + .setMaxIter(10) \ + .setRegParam(0.01) pipeline = Pipeline() \ - .setStages([tokenizer, hashingTF, lr]) + .setStages([tokenizer, hashingTF, lr]) model = pipeline.fit(training) @@ -50,7 +50,7 @@ sc.parallelize([(4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ + (7L, "apache hadoop")]) .map(lambda x: Row(id=x[0], text=x[1]))) for row in model.transform(test).collect(): diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 1cf9d3065f3d1..3b2ab269d9f21 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -18,7 +18,7 @@ from abc import ABCMeta, abstractmethod, abstractproperty from pyspark import SparkContext -from pyspark.sql import SchemaRDD, inherit_doc # TODO: move inherit_doc to Spark Core +from pyspark.sql import SchemaRDD, inherit_doc # TODO: move inherit_doc to Spark Core from pyspark.ml.param import Param, Params from pyspark.ml.util import Identifiable From d3e8dbe9205c40a7a48f0577a09c8e3406b106c2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 26 Jan 2015 13:10:17 -0800 Subject: [PATCH 10/28] more docs optimize pipeline.fit impl --- python/pyspark/ml/__init__.py | 117 ++++++++++++++++++++-------- python/pyspark/ml/classification.py | 1 + python/pyspark/ml/param/__init__.py | 6 -- 3 files changed, 86 insertions(+), 38 deletions(-) diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 3b2ab269d9f21..d1cc4d621a657 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -26,21 +26,15 @@ def _jvm(): - return SparkContext._jvm - - -def _inherit_doc(cls): - for name, func in vars(cls).items(): - # only inherit docstring for public functions - if name.startswith("_"): - continue - if not func.__doc__: - for parent in cls.__bases__: - parent_func = getattr(parent, name, None) - if parent_func and getattr(parent_func, "__doc__", None): - func.__doc__ = parent_func.__doc__ - break - return cls + """ + Returns the JVM view associated with SparkContext. Must be called + after SparkContext is initialized. + """ + jvm = SparkContext._jvm + if jvm: + return jvm + else: + raise AttributeError("Cannot load _jvm from SparkContext. Is SparkContext initialized?") @inherit_doc @@ -50,6 +44,8 @@ class PipelineStage(Params): :py:class:`Transformer`. """ + __metaclass__ = ABCMeta + def __init__(self): super(PipelineStage, self).__init__() @@ -147,38 +143,54 @@ def getStages(self): return self.paramMap[self.stages] def fit(self, dataset, params={}): - map = self._merge_params(params) - transformers = [] - for stage in self.getStages(): - if isinstance(stage, Transformer): - transformers.append(stage) - dataset = stage.transform(dataset, map) - elif isinstance(stage, Estimator): - model = stage.fit(dataset, map) - transformers.append(model) - dataset = model.transform(dataset, map) - else: + paramMap = self._merge_params(params) + stages = paramMap(self.stages) + for stage in stages: + if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): raise ValueError( "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) + indexOfLastEstimator = -1 + for i, stage in enumerate(stages): + if isinstance(stage, Estimator): + indexOfLastEstimator = i + transformers = [] + for i, stage in enumerate(stages): + if i <= indexOfLastEstimator: + if isinstance(stage, Transformer): + transformers.append(stage) + dataset = stage.transform(dataset, paramMap) + else: # must be an Estimator + model = stage.fit(dataset, paramMap) + transformers.append(model) + dataset = model.transform(dataset, paramMap) + else: + transformers.append(stage) return PipelineModel(transformers) @inherit_doc class PipelineModel(Transformer): + """ + Represents a compiled pipeline with transformers and fitted models. + """ def __init__(self, transformers): super(PipelineModel, self).__init__() self.transformers = transformers def transform(self, dataset, params={}): - map = self._merge_params(params) + paramMap = self._merge_params(params) for t in self.transformers: - dataset = t.transform(dataset, map) + dataset = t.transform(dataset, paramMap) return dataset @inherit_doc -class JavaWrapper(object): +class JavaWrapper(Params): + """ + Utility class to help create wrapper classes from Java/Scala + implementations of pipeline components. + """ __metaclass__ = ABCMeta @@ -187,17 +199,45 @@ def __init__(self): @abstractproperty def _java_class(self): + """ + Fully-qualified class name of the wrapped Java component. + """ raise NotImplementedError def _create_java_obj(self): + """ + Creates a new Java object and returns its reference. + """ java_obj = _jvm() for name in self._java_class.split("."): java_obj = getattr(java_obj, name) return java_obj() + def _transfer_params_to_java(self, params, java_obj): + """ + Transforms the embedded params and additional params to the + input Java object. + :param params: additional params (overwriting embedded values) + :param java_obj: Java object to receive the params + """ + paramMap = self._merge_params(params) + for param in self.params(): + if param in paramMap: + java_obj.set(param.name, paramMap[param]) + + def _empty_java_param_map(self): + """ + Returns an empty Java ParamMap reference. + """ + return _jvm().org.apache.spark.ml.param.ParamMap() + @inherit_doc class JavaEstimator(Estimator, JavaWrapper): + """ + Base class for :py:class:`Estimator`s that wrap Java/Scala + implementations. + """ __metaclass__ = ABCMeta @@ -206,12 +246,22 @@ def __init__(self): @abstractmethod def _create_model(self, java_model): + """ + Creates a model from the input Java model reference. + """ raise NotImplementedError def _fit_java(self, dataset, params={}): + """ + Fits a Java model to the input dataset. + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: additional params (overwriting embedded values) + :return: fitted Java model + """ java_obj = self._create_java_obj() self._transfer_params_to_java(params, java_obj) - return java_obj.fit(dataset._jschema_rdd, _jvm().org.apache.spark.ml.param.ParamMap()) + return java_obj.fit(dataset._jschema_rdd, self._empty_java_param_map()) def fit(self, dataset, params={}): java_model = self._fit_java(dataset, params) @@ -220,6 +270,10 @@ def fit(self, dataset, params={}): @inherit_doc class JavaTransformer(Transformer, JavaWrapper): + """ + Base class for :py:class:`Transformer`s that wrap Java/Scala + implementations. + """ __metaclass__ = ABCMeta @@ -229,6 +283,5 @@ def __init__(self): def transform(self, dataset, params={}): java_obj = self._create_java_obj() self._transfer_params_to_java(params, java_obj) - return SchemaRDD(java_obj.transform(dataset._jschema_rdd, - _jvm().org.apache.spark.ml.param.ParamMap()), + return SchemaRDD(java_obj.transform(dataset._jschema_rdd, self._empty_java_param_map()), dataset.sql_ctx) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 3a105f1a10a3d..ab29fb3220a63 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -46,6 +46,7 @@ class LogisticRegressionModel(Transformer): """ def __init__(self, java_model): + super(LogisticRegressionModel, self).__init__() self._java_model = java_model def transform(self, dataset, params={}): diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index f81b836a2242f..037a6fa2000bc 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -68,9 +68,3 @@ def _merge_params(self, params): map = self.paramMap.copy() map.update(params) return map - - def _transfer_params_to_java(self, params, java_obj): - map = self._merge_params(params) - for param in self.params(): - if param in map: - java_obj.set(param.name, map[param]) From 05e3e40fd040c24f0b5be8c1b544e6bb863009d9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 26 Jan 2015 13:19:51 -0800 Subject: [PATCH 11/28] update example --- .../ml/simple_text_classification_pipeline.py | 15 ++++++++++++++- python/pyspark/ml/__init__.py | 2 +- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index f457ce97a42d1..3160d8452f848 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -22,6 +22,15 @@ from pyspark.ml.classification import LogisticRegression +""" +A simple text classification pipeline that recognizes "spark" from +input text. This is to show how to create and configure a Spark ML +pipeline in Python. Run with: + + bin/spark-submit examples/src/main/python/ml/simple_text_classification_pipeline.py +""" + + if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") sqlCtx = SQLContext(sc) @@ -53,5 +62,9 @@ (7L, "apache hadoop")]) .map(lambda x: Row(id=x[0], text=x[1]))) - for row in model.transform(test).collect(): + prediction = model.transform(test) + + prediction.registerTempTable("prediction") + selected = sqlCtx.sql("SELECT id, text, prediction from prediction") + for row in selected.collect(): print row diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index d1cc4d621a657..aaa0b2b73e8d1 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -144,7 +144,7 @@ def getStages(self): def fit(self, dataset, params={}): paramMap = self._merge_params(params) - stages = paramMap(self.stages) + stages = paramMap[self.stages] for stage in stages: if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): raise ValueError( From f4d0fe6c82ca821c05dab101807f37637ccafa30 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 10:05:51 -0800 Subject: [PATCH 12/28] use LabeledDocument and Document in example --- .../main/python/ml/simple_text_classification_pipeline.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index 3160d8452f848..014624962a541 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -34,12 +34,13 @@ if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") sqlCtx = SQLContext(sc) + LabeledDocument = Row('id', 'text', 'label') training = sqlCtx.inferSchema( sc.parallelize([(0L, "a b c d e spark", 1.0), (1L, "b d", 0.0), (2L, "spark f g h", 1.0), (3L, "hadoop mapreduce", 0.0)]) - .map(lambda x: Row(id=x[0], text=x[1], label=x[2]))) + .map(lambda x: LabeledDocument(*x))) tokenizer = Tokenizer() \ .setInputCol("text") \ @@ -55,12 +56,13 @@ model = pipeline.fit(training) + Document = Row('id', 'text') test = sqlCtx.inferSchema( sc.parallelize([(4L, "spark i j k"), (5L, "l m n"), (6L, "mapreduce spark"), (7L, "apache hadoop")]) - .map(lambda x: Row(id=x[0], text=x[1]))) + .map(lambda x: Document(*x))) prediction = model.transform(test) From d5efd34739d45d84799f514c0a84a485076fccd1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 10:12:04 -0800 Subject: [PATCH 13/28] update doc conf and move embedded param map to instance attribute --- python/docs/pyspark.ml.rst | 5 ++++- python/pyspark/ml/param/__init__.py | 11 +++++------ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index f4c1a6406e020..064fcd9c03204 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -11,6 +11,7 @@ pyspark.ml module :members: :undoc-members: :show-inheritance: + :inherited-members: pyspark.ml.param module ----------------------- @@ -19,6 +20,7 @@ pyspark.ml.param module :members: :undoc-members: :show-inheritance: + :inherited-members: pyspark.ml.feature module ------------------------- @@ -27,6 +29,7 @@ pyspark.ml.feature module :members: :undoc-members: :show-inheritance: + :inherited-members: pyspark.ml.classification module -------------------------------- @@ -34,5 +37,5 @@ pyspark.ml.classification module .. automodule:: pyspark.ml.classification :members: :undoc-members: - :inherited-members: :show-inheritance: + :inherited-members: diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 037a6fa2000bc..1d561f876a0fe 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -50,11 +50,10 @@ class Params(Identifiable): __metaclass__ = ABCMeta - #: Internal param map. - paramMap = {} - def __init__(self): super(Params, self).__init__() + #: embedded param map + self.paramMap = {} def params(self): """ @@ -65,6 +64,6 @@ def params(self): return filter(lambda x: isinstance(x, Param), map(lambda x: getattr(self, x), dir(self))) def _merge_params(self, params): - map = self.paramMap.copy() - map.update(params) - return map + paramMap = self.paramMap.copy() + paramMap.update(params) + return paramMap From f66ba0c6b15ad57975e7e2282e02a1687b78b0f4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 10:27:12 -0800 Subject: [PATCH 14/28] make params a property --- python/pyspark/ml/__init__.py | 2 +- python/pyspark/ml/param/__init__.py | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index aaa0b2b73e8d1..e16214ae18431 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -221,7 +221,7 @@ def _transfer_params_to_java(self, params, java_obj): :param java_obj: Java object to receive the params """ paramMap = self._merge_params(params) - for param in self.params(): + for param in self.params: if param in paramMap: java_obj.set(param.name, paramMap[param]) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 1d561f876a0fe..594fc9eae7382 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -55,13 +55,15 @@ def __init__(self): #: embedded param map self.paramMap = {} + @property def params(self): """ Returns all params. The default implementation uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ - return filter(lambda x: isinstance(x, Param), map(lambda x: getattr(self, x), dir(self))) + return filter(lambda attr: isinstance(attr, Param), + [getattr(self, x) for x in dir(self) if x != "params"]) def _merge_params(self, params): paramMap = self.paramMap.copy() From 1dcc17e222be2da865ab3c016a96979f77728305 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 13:43:19 -0800 Subject: [PATCH 15/28] update code gen and make param appear in the doc --- python/pyspark/ml/param/__init__.py | 12 ++- python/pyspark/ml/param/_gen_shared_params.py | 43 +++++--- python/pyspark/ml/param/shared.py | 98 +++++++++++++++++-- 3 files changed, 132 insertions(+), 21 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 594fc9eae7382..298ac20099f6b 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -39,7 +39,8 @@ def __str__(self): return str(self.parent) + "_" + self.name def __repr__(self): - return str(self.parent) + "_" + self.name + return "Param(parent=%r, name=%r, doc=%r, defaultValue=%r)" % \ + (self.parent, self.name, self.doc, self.defaultValue) class Params(Identifiable): @@ -69,3 +70,12 @@ def _merge_params(self, params): paramMap = self.paramMap.copy() paramMap.update(params) return paramMap + + @staticmethod + def _dummy(): + """ + Returns a dummy Params instance used as a placeholder to generate docs. + """ + dummy = Params() + dummy.uid = "undefined" + return dummy diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_gen_shared_params.py index f40823b906221..30e141698e5c9 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_gen_shared_params.py @@ -42,25 +42,42 @@ def _gen_param_code(name, doc, defaultValue): :param defaultValue: string representation of the param :return: code string """ - upperCamelName = name[0].upper() + name[1:] - return """class Has%s(Params): + # TODO: How to correctly inherit instance attributes? + template = '''class Has$Name(Params): + """ + Params with $name. + """ + + # a placeholder to make it appear in the generated doc + $name = Param(Params._dummy(), "$name", "$doc", $defaultValue) def __init__(self): - super(Has%s, self).__init__() - #: %s - self.%s = Param(self, "%s", "%s", %s) + super(Has$Name, self).__init__() + #: param for $doc + self.$name = Param(self, "$name", "$doc", $defaultValue) - def set%s(self, value): - self.paramMap[self.%s] = value + def set$Name(self, value): + """ + Sets the value of :py:attr:`$name`. + """ + self.paramMap[self.$name] = value return self - def get%s(self): - if self.%s in self.paramMap: - return self.paramMap[self.%s] + def get$Name(self): + """ + Gets the value of $name or its default value. + """ + if self.$name in self.paramMap: + return self.paramMap[self.$name] else: - return self.%s.defaultValue""" % ( - upperCamelName, upperCamelName, doc, name, name, doc, defaultValue, upperCamelName, name, - upperCamelName, name, name, name) + return self.$name.defaultValue''' + + upperCamelName = name[0].upper() + name[1:] + return template \ + .replace("$name", name) \ + .replace("$Name", upperCamelName) \ + .replace("$doc", doc) \ + .replace("$defaultValue", defaultValue) if __name__ == "__main__": print header diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 8680f389577b6..94ebade50ea47 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -21,17 +21,29 @@ class HasMaxIter(Params): + """ + Params with maxIter. + """ + + # a placeholder to make it appear in the generated doc + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations", 100) def __init__(self): super(HasMaxIter, self).__init__() - #: max number of iterations + #: param for max number of iterations self.maxIter = Param(self, "maxIter", "max number of iterations", 100) def setMaxIter(self, value): + """ + Sets the value of :py:attr:`maxIter`. + """ self.paramMap[self.maxIter] = value return self def getMaxIter(self): + """ + Gets the value of maxIter or its default value. + """ if self.maxIter in self.paramMap: return self.paramMap[self.maxIter] else: @@ -39,17 +51,29 @@ def getMaxIter(self): class HasRegParam(Params): + """ + Params with regParam. + """ + + # a placeholder to make it appear in the generated doc + regParam = Param(Params._dummy(), "regParam", "regularization constant", 0.1) def __init__(self): super(HasRegParam, self).__init__() - #: regularization constant + #: param for regularization constant self.regParam = Param(self, "regParam", "regularization constant", 0.1) def setRegParam(self, value): + """ + Sets the value of :py:attr:`regParam`. + """ self.paramMap[self.regParam] = value return self def getRegParam(self): + """ + Gets the value of regParam or its default value. + """ if self.regParam in self.paramMap: return self.paramMap[self.regParam] else: @@ -57,17 +81,29 @@ def getRegParam(self): class HasFeaturesCol(Params): + """ + Params with featuresCol. + """ + + # a placeholder to make it appear in the generated doc + featuresCol = Param(Params._dummy(), "featuresCol", "features column name", 'features') def __init__(self): super(HasFeaturesCol, self).__init__() - #: features column name + #: param for features column name self.featuresCol = Param(self, "featuresCol", "features column name", 'features') def setFeaturesCol(self, value): + """ + Sets the value of :py:attr:`featuresCol`. + """ self.paramMap[self.featuresCol] = value return self def getFeaturesCol(self): + """ + Gets the value of featuresCol or its default value. + """ if self.featuresCol in self.paramMap: return self.paramMap[self.featuresCol] else: @@ -75,17 +111,29 @@ def getFeaturesCol(self): class HasLabelCol(Params): + """ + Params with labelCol. + """ + + # a placeholder to make it appear in the generated doc + labelCol = Param(Params._dummy(), "labelCol", "label column name", 'label') def __init__(self): super(HasLabelCol, self).__init__() - #: label column name + #: param for label column name self.labelCol = Param(self, "labelCol", "label column name", 'label') def setLabelCol(self, value): + """ + Sets the value of :py:attr:`labelCol`. + """ self.paramMap[self.labelCol] = value return self def getLabelCol(self): + """ + Gets the value of labelCol or its default value. + """ if self.labelCol in self.paramMap: return self.paramMap[self.labelCol] else: @@ -93,17 +141,29 @@ def getLabelCol(self): class HasPredictionCol(Params): + """ + Params with predictionCol. + """ + + # a placeholder to make it appear in the generated doc + predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name", 'prediction') def __init__(self): super(HasPredictionCol, self).__init__() - #: prediction column name + #: param for prediction column name self.predictionCol = Param(self, "predictionCol", "prediction column name", 'prediction') def setPredictionCol(self, value): + """ + Sets the value of :py:attr:`predictionCol`. + """ self.paramMap[self.predictionCol] = value return self def getPredictionCol(self): + """ + Gets the value of predictionCol or its default value. + """ if self.predictionCol in self.paramMap: return self.paramMap[self.predictionCol] else: @@ -111,17 +171,29 @@ def getPredictionCol(self): class HasInputCol(Params): + """ + Params with inputCol. + """ + + # a placeholder to make it appear in the generated doc + inputCol = Param(Params._dummy(), "inputCol", "input column name", 'input') def __init__(self): super(HasInputCol, self).__init__() - #: input column name + #: param for input column name self.inputCol = Param(self, "inputCol", "input column name", 'input') def setInputCol(self, value): + """ + Sets the value of :py:attr:`inputCol`. + """ self.paramMap[self.inputCol] = value return self def getInputCol(self): + """ + Gets the value of inputCol or its default value. + """ if self.inputCol in self.paramMap: return self.paramMap[self.inputCol] else: @@ -129,17 +201,29 @@ def getInputCol(self): class HasOutputCol(Params): + """ + Params with outputCol. + """ + + # a placeholder to make it appear in the generated doc + outputCol = Param(Params._dummy(), "outputCol", "output column name", 'output') def __init__(self): super(HasOutputCol, self).__init__() - #: output column name + #: param for output column name self.outputCol = Param(self, "outputCol", "output column name", 'output') def setOutputCol(self, value): + """ + Sets the value of :py:attr:`outputCol`. + """ self.paramMap[self.outputCol] = value return self def getOutputCol(self): + """ + Gets the value of outputCol or its default value. + """ if self.outputCol in self.paramMap: return self.paramMap[self.outputCol] else: From 46fa1471ebacdb7c76c2ef6872ef92d1f4f073f6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 13:45:00 -0800 Subject: [PATCH 16/28] update mllib/pom.xml to include python files in the assembly --- mllib/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/mllib/pom.xml b/mllib/pom.xml index a0bda89ccaa71..7b7beaf59d331 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -125,6 +125,8 @@ ../python pyspark/mllib/*.py + pyspark/ml/*.py + pyspark/ml/param/*.py From 036ca04800b40c6a1953036c722b5aa6edd9494c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 13:51:24 -0800 Subject: [PATCH 17/28] gen numFeatures --- python/pyspark/ml/feature.py | 17 ++--------- python/pyspark/ml/param/__init__.py | 2 +- python/pyspark/ml/param/_gen_shared_params.py | 3 +- python/pyspark/ml/param/shared.py | 30 +++++++++++++++++++ 4 files changed, 35 insertions(+), 17 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 9f3b1c7a055cb..8109d4097e616 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -17,8 +17,7 @@ from pyspark.sql import inherit_doc from pyspark.ml import JavaTransformer -from pyspark.ml.param import Param -from pyspark.ml.param.shared import HasInputCol, HasOutputCol +from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures @inherit_doc @@ -33,23 +32,11 @@ def _java_class(self): @inherit_doc -class HashingTF(JavaTransformer, HasInputCol, HasOutputCol): +class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): def __init__(self): super(HashingTF, self).__init__() - #: param for number of features - self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) @property def _java_class(self): return "org.apache.spark.ml.feature.HashingTF" - - def setNumFeatures(self, value): - self.paramMap[self.numFeatures] = value - return self - - def getNumFeatures(self): - if self.numFeatures in self.paramMap: - return self.paramMap[self.numFeatures] - else: - return self.numFeatures.defaultValue diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 298ac20099f6b..80c62c8cdb5d0 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -36,7 +36,7 @@ def __init__(self, parent, name, doc, defaultValue=None): self.defaultValue = defaultValue def __str__(self): - return str(self.parent) + "_" + self.name + return str(self.parent) + "-" + self.name def __repr__(self): return "Param(parent=%r, name=%r, doc=%r, defaultValue=%r)" % \ diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_gen_shared_params.py index 30e141698e5c9..5eb81106f116c 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_gen_shared_params.py @@ -90,7 +90,8 @@ def get$Name(self): ("labelCol", "label column name", "'label'"), ("predictionCol", "prediction column name", "'prediction'"), ("inputCol", "input column name", "'input'"), - ("outputCol", "output column name", "'output'")] + ("outputCol", "output column name", "'output'"), + ("numFeatures", "number of features", "1 << 18")] code = [] for name, doc, defaultValue in shared: code.append(_gen_param_code(name, doc, defaultValue)) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 94ebade50ea47..586822f2de423 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -228,3 +228,33 @@ def getOutputCol(self): return self.paramMap[self.outputCol] else: return self.outputCol.defaultValue + + +class HasNumFeatures(Params): + """ + Params with numFeatures. + """ + + # a placeholder to make it appear in the generated doc + numFeatures = Param(Params._dummy(), "numFeatures", "number of features", 1 << 18) + + def __init__(self): + super(HasNumFeatures, self).__init__() + #: param for number of features + self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) + + def setNumFeatures(self, value): + """ + Sets the value of :py:attr:`numFeatures`. + """ + self.paramMap[self.numFeatures] = value + return self + + def getNumFeatures(self): + """ + Gets the value of numFeatures or its default value. + """ + if self.numFeatures in self.paramMap: + return self.paramMap[self.numFeatures] + else: + return self.numFeatures.defaultValue From 5153cff555d5188dc9a2844fe558e73af38fbd01 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 14:12:21 -0800 Subject: [PATCH 18/28] simplify java models --- .../org/apache/spark/ml/param/params.scala | 1 - python/pyspark/ml/__init__.py | 50 ++++++++++++++++--- python/pyspark/ml/classification.py | 14 +++--- 3 files changed, 49 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 33f7a3900a98e..5fb4379e23c2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -293,7 +293,6 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten new ParamMap(this.map ++ other.map) } - /** * Adds all parameters from the input param map into this param map. */ diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index e16214ae18431..a193442841f65 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -101,6 +101,18 @@ def transform(self, dataset, params={}): raise NotImplementedError() +@inherit_doc +class Model(Transformer): + """ + Abstract class for models fitted by :py:class:`Estimator`s. + """ + + ___metaclass__ = ABCMeta + + def __init__(self): + super(Model, self).__init__() + + @inherit_doc class Pipeline(Estimator): """ @@ -169,7 +181,7 @@ def fit(self, dataset, params={}): @inherit_doc -class PipelineModel(Transformer): +class PipelineModel(Model): """ Represents a compiled pipeline with transformers and fitted models. """ @@ -204,9 +216,9 @@ def _java_class(self): """ raise NotImplementedError - def _create_java_obj(self): + def _java_obj(self): """ - Creates a new Java object and returns its reference. + Returns or creates a Java object. """ java_obj = _jvm() for name in self._java_class.split("."): @@ -231,6 +243,13 @@ def _empty_java_param_map(self): """ return _jvm().org.apache.spark.ml.param.ParamMap() + def _create_java_param_map(self, params, java_obj): + paramMap = self._empty_java_param_map() + for param, value in params.items(): + if param.parent is self: + paramMap.put(java_obj.getParam(param.name), value) + return paramMap + @inherit_doc class JavaEstimator(Estimator, JavaWrapper): @@ -259,7 +278,7 @@ def _fit_java(self, dataset, params={}): :param params: additional params (overwriting embedded values) :return: fitted Java model """ - java_obj = self._create_java_obj() + java_obj = self._java_obj() self._transfer_params_to_java(params, java_obj) return java_obj.fit(dataset._jschema_rdd, self._empty_java_param_map()) @@ -281,7 +300,24 @@ def __init__(self): super(JavaTransformer, self).__init__() def transform(self, dataset, params={}): - java_obj = self._create_java_obj() - self._transfer_params_to_java(params, java_obj) - return SchemaRDD(java_obj.transform(dataset._jschema_rdd, self._empty_java_param_map()), + java_obj = self._java_obj() + self._transfer_params_to_java({}, java_obj) + java_param_map = self._create_java_param_map(params, java_obj) + return SchemaRDD(java_obj.transform(dataset._jschema_rdd, java_param_map), dataset.sql_ctx) + + +@inherit_doc +class JavaModel(JavaTransformer): + """ + Base class for :py:class:`Model`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def __init__(self): + super(JavaTransformer, self).__init__() + + def _java_obj(self): + return self._java_model diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index ab29fb3220a63..4628cef6e255c 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -15,8 +15,8 @@ # limitations under the License. # -from pyspark.sql import SchemaRDD, inherit_doc -from pyspark.ml import JavaEstimator, Transformer, _jvm +from pyspark.sql import inherit_doc +from pyspark.ml import JavaEstimator, JavaModel from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ HasRegParam @@ -40,7 +40,7 @@ def _create_model(self, java_model): @inherit_doc -class LogisticRegressionModel(Transformer): +class LogisticRegressionModel(JavaModel): """ Model fitted by LogisticRegression. """ @@ -49,8 +49,6 @@ def __init__(self, java_model): super(LogisticRegressionModel, self).__init__() self._java_model = java_model - def transform(self, dataset, params={}): - # TODO: handle params here. - return SchemaRDD(self._java_model.transform( - dataset._jschema_rdd, - _jvm().org.apache.spark.ml.param.ParamMap()), dataset.sql_ctx) + @property + def _java_class(self): + return "org.apache.spark.ml.classification.LogisticRegressionModel" From 0586c7b0b575cb9f261c5c3c1a20927bf9077f6d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 14:15:20 -0800 Subject: [PATCH 19/28] add more comments to the example --- .../python/ml/simple_text_classification_pipeline.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index 014624962a541..c7df3d7b74767 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -34,6 +34,8 @@ if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") sqlCtx = SQLContext(sc) + + # Prepare training documents, which are labeled. LabeledDocument = Row('id', 'text', 'label') training = sqlCtx.inferSchema( sc.parallelize([(0L, "a b c d e spark", 1.0), @@ -42,6 +44,7 @@ (3L, "hadoop mapreduce", 0.0)]) .map(lambda x: LabeledDocument(*x))) + # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer() \ .setInputCol("text") \ .setOutputCol("words") @@ -54,8 +57,10 @@ pipeline = Pipeline() \ .setStages([tokenizer, hashingTF, lr]) + # Fit the pipeline to training documents. model = pipeline.fit(training) + # Prepare test documents, which are unlabeled. Document = Row('id', 'text') test = sqlCtx.inferSchema( sc.parallelize([(4L, "spark i j k"), @@ -64,9 +69,11 @@ (7L, "apache hadoop")]) .map(lambda x: Document(*x))) + # Make predictions on test documents and print columns of interest. prediction = model.transform(test) - prediction.registerTempTable("prediction") selected = sqlCtx.sql("SELECT id, text, prediction from prediction") for row in selected.collect(): print row + + sc.stop() From ba0ba1e2b0fce9418ca8ff2afbf31a8f3018e61c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 14:57:31 -0800 Subject: [PATCH 20/28] add unit tests for pipeline --- python/pyspark/ml/__init__.py | 3 +- python/pyspark/ml/tests.py | 115 ++++++++++++++++++++++++++++++++++ 2 files changed, 117 insertions(+), 1 deletion(-) create mode 100644 python/pyspark/ml/tests.py diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index a193442841f65..5b484f7392521 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -174,7 +174,8 @@ def fit(self, dataset, params={}): else: # must be an Estimator model = stage.fit(dataset, paramMap) transformers.append(model) - dataset = model.transform(dataset, paramMap) + if i < indexOfLastEstimator: + dataset = model.transform(dataset, paramMap) else: transformers.append(stage) return PipelineModel(transformers) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py new file mode 100644 index 0000000000000..2f5f2d18d5d01 --- /dev/null +++ b/python/pyspark/ml/tests.py @@ -0,0 +1,115 @@ +# +# 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. +# + +""" +Unit tests for Spark ML Python APIs. +""" + +import sys + +if sys.version_info[:2] <= (2, 6): + try: + import unittest2 as unittest + except ImportError: + sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier') + sys.exit(1) +else: + import unittest + +from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase +from pyspark.sql import SchemaRDD +from pyspark.ml import Transformer, Estimator, Model, Pipeline +from pyspark.ml.param import Param + + +class MockDataset(SchemaRDD): + + def __init__(self): + self.index = 0 + + +class MockTransformer(Transformer): + + def __init__(self): + super(MockTransformer, self).__init__() + self.fake = Param(self, "fake", "fake", None) + self.dataset_index = None + self.fake_param_value = None + + def transform(self, dataset, params={}): + self.dataset_index = dataset.index + if self.fake in params: + self.fake_param_value = params[self.fake] + dataset.index += 1 + return dataset + + +class MockEstimator(Estimator): + + def __init__(self): + super(MockEstimator, self).__init__() + self.fake = Param(self, "fake", "fake", None) + self.dataset_index = None + self.fake_param_value = None + self.model = None + + def fit(self, dataset, params={}): + self.dataset_index = dataset.index + if self.fake in params: + self.fake_param_value = params[self.fake] + model = MockModel() + self.model = model + return model + + +class MockModel(MockTransformer, Model): + + def __init__(self): + super(MockModel, self).__init__() + + +class PipelineTests(PySparkTestCase): + + def test_pipeline(self): + dataset = MockDataset() + estimator0 = MockEstimator() + transformer1 = MockTransformer() + estimator2 = MockEstimator() + transformer3 = MockTransformer() + pipeline = Pipeline() \ + .setStages([estimator0, transformer1, estimator2, transformer3]) + pipeline_model = pipeline.fit(dataset, {estimator0.fake: 0, transformer1.fake: 1}) + self.assertEqual(0, estimator0.dataset_index) + self.assertEqual(0, estimator0.fake_param_value) + model0 = estimator0.model + self.assertEqual(0, model0.dataset_index) + self.assertEqual(1, transformer1.dataset_index) + self.assertEqual(1, transformer1.fake_param_value) + self.assertEqual(2, estimator2.dataset_index) + model2 = estimator2.model + self.assertIsNone(model2.dataset_index, "The model produced by the last estimator should " + "not be called during fit.") + dataset = pipeline_model.transform(dataset) + self.assertEqual(2, model0.dataset_index) + self.assertEqual(3, transformer1.dataset_index) + self.assertEqual(4, model2.dataset_index) + self.assertEqual(5, transformer3.dataset_index) + self.assertEqual(6, dataset.index) + + +if __name__ == "__main__": + unittest.main() From 7521d1c656f5fd8fadb7c87912d710a8bdacd280 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 15:14:26 -0800 Subject: [PATCH 21/28] add unit tests to HashingTF and Tokenizer --- python/pyspark/ml/feature.py | 46 ++++++++++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 8109d4097e616..dee574774e83e 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -22,6 +22,19 @@ @inherit_doc class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): + """ + A tokenizer that converts the input string to lowercase and then splits it by white spaces. + + >>> from pyspark.sql import Row + >>> dataset = sqlCtx.inferSchema(sc.parallelize([Row(text="a b c")])) + >>> tokenizer = Tokenizer() \ + .setInputCol("text") \ + .setOutputCol("words") + >>> print tokenizer.transform(dataset).first() + Row(text=u'a b c', words=[u'a', u'b', u'c']) + >>> print tokenizer.transform(dataset, {tokenizer.outputCol: "tokens"}).first() + Row(text=u'a b c', tokens=[u'a', u'b', u'c']) + """ def __init__(self): super(Tokenizer, self).__init__() @@ -33,6 +46,21 @@ def _java_class(self): @inherit_doc class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): + """ + Maps a sequence of terms to their term frequencies using the hashing trick. + + >>> from pyspark.sql import Row + >>> dataset = sqlCtx.inferSchema(sc.parallelize([Row(words=["a", "b", "c"])])) + >>> hashingTF = HashingTF() \ + .setNumFeatures(10) \ + .setInputCol("words") \ + .setOutputCol("features") + >>> print hashingTF.transform(dataset).first().features + (10,[7,8,9],[1.0,1.0,1.0]) + >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} + >>> print hashingTF.transform(dataset, params).first().vector + (5,[2,3,4],[1.0,1.0,1.0]) + """ def __init__(self): super(HashingTF, self).__init__() @@ -40,3 +68,21 @@ def __init__(self): @property def _java_class(self): return "org.apache.spark.ml.feature.HashingTF" + + +if __name__ == "__main__": + import doctest + from pyspark.context import SparkContext + from pyspark.sql import Row, SQLContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + sc = SparkContext("local[2]", "ml.feature tests") + sqlCtx = SQLContext(sc) + globs['sc'] = sc + globs['sqlCtx'] = sqlCtx + (failure_count, test_count) = doctest.testmod( + globs=globs, optionflags=doctest.ELLIPSIS) + sc.stop() + if failure_count: + exit(-1) From a4f4dbf146235b6dd7bf216d5b903c6dddc8503b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 15:25:36 -0800 Subject: [PATCH 22/28] add unit test for LR --- python/pyspark/ml/classification.py | 34 +++++++++++++++++++++++++++++ python/pyspark/ml/feature.py | 2 +- 2 files changed, 35 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4628cef6e255c..0384c39b5d5ab 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -26,6 +26,22 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti HasRegParam): """ Logistic regression. + + >>> from pyspark.sql import Row + >>> from pyspark.mllib.linalg import Vectors + >>> dataset = sqlCtx.inferSchema(sc.parallelize([ \ + Row(label=1.0, features=Vectors.dense(1.0)), \ + Row(label=0.0, features=Vectors.sparse(1, [], []))])) + >>> lr = LogisticRegression() \ + .setMaxIter(5) \ + .setRegParam(0.01) + >>> model = lr.fit(dataset) + >>> test0 = sqlCtx.inferSchema(sc.parallelize([Row(features=Vectors.dense(-1.0))])) + >>> print model.transform(test0).first().prediction + 0.0 + >>> test1 = sqlCtx.inferSchema(sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))])) + >>> print model.transform(test1).first().prediction + 1.0 """ def __init__(self): @@ -52,3 +68,21 @@ def __init__(self, java_model): @property def _java_class(self): return "org.apache.spark.ml.classification.LogisticRegressionModel" + + +if __name__ == "__main__": + import doctest + from pyspark.context import SparkContext + from pyspark.sql import SQLContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + sc = SparkContext("local[2]", "ml.feature tests") + sqlCtx = SQLContext(sc) + globs['sc'] = sc + globs['sqlCtx'] = sqlCtx + (failure_count, test_count) = doctest.testmod( + globs=globs, optionflags=doctest.ELLIPSIS) + sc.stop() + if failure_count: + exit(-1) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index dee574774e83e..45191c8fcb3dc 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -73,7 +73,7 @@ def _java_class(self): if __name__ == "__main__": import doctest from pyspark.context import SparkContext - from pyspark.sql import Row, SQLContext + from pyspark.sql import SQLContext globs = globals().copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: From 0882513f9b394c4812472a69f471a10c179517e8 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 27 Jan 2015 15:54:07 -0800 Subject: [PATCH 23/28] update doc style --- python/pyspark/ml/feature.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 45191c8fcb3dc..7e9746cdccac2 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -23,7 +23,8 @@ @inherit_doc class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): """ - A tokenizer that converts the input string to lowercase and then splits it by white spaces. + A tokenizer that converts the input string to lowercase and then + splits it by white spaces. >>> from pyspark.sql import Row >>> dataset = sqlCtx.inferSchema(sc.parallelize([Row(text="a b c")])) @@ -47,7 +48,8 @@ def _java_class(self): @inherit_doc class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): """ - Maps a sequence of terms to their term frequencies using the hashing trick. + Maps a sequence of terms to their term frequencies using the + hashing trick. >>> from pyspark.sql import Row >>> dataset = sqlCtx.inferSchema(sc.parallelize([Row(words=["a", "b", "c"])])) From 1dca16a674d412611ea181167168168b18557900 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 27 Jan 2015 18:05:51 -0800 Subject: [PATCH 24/28] refactor --- python/docs/pyspark.rst | 1 + python/pyspark/ml/__init__.py | 309 +--------------------------- python/pyspark/ml/classification.py | 23 +-- python/pyspark/ml/feature.py | 20 +- python/pyspark/ml/param/__init__.py | 2 +- python/pyspark/ml/pipeline.py | 153 ++++++++++++++ python/pyspark/ml/tests.py | 4 +- python/pyspark/ml/util.py | 148 ++++++++++++- python/pyspark/sql.py | 14 -- 9 files changed, 316 insertions(+), 358 deletions(-) create mode 100644 python/pyspark/ml/pipeline.py diff --git a/python/docs/pyspark.rst b/python/docs/pyspark.rst index e81be3b6cb796..0df12c49ad033 100644 --- a/python/docs/pyspark.rst +++ b/python/docs/pyspark.rst @@ -9,6 +9,7 @@ Subpackages pyspark.sql pyspark.streaming + pyspark.ml pyspark.mllib Contents diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 5b484f7392521..0a065b3155e1a 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -15,310 +15,7 @@ # limitations under the License. # -from abc import ABCMeta, abstractmethod, abstractproperty +from pyspark.ml.param import * +from pyspark.ml.pipeline import * -from pyspark import SparkContext -from pyspark.sql import SchemaRDD, inherit_doc # TODO: move inherit_doc to Spark Core -from pyspark.ml.param import Param, Params -from pyspark.ml.util import Identifiable - -__all__ = ["Pipeline", "Transformer", "Estimator", "param", "feature", "classification"] - - -def _jvm(): - """ - Returns the JVM view associated with SparkContext. Must be called - after SparkContext is initialized. - """ - jvm = SparkContext._jvm - if jvm: - return jvm - else: - raise AttributeError("Cannot load _jvm from SparkContext. Is SparkContext initialized?") - - -@inherit_doc -class PipelineStage(Params): - """ - A stage in a pipeline, either an :py:class:`Estimator` or a - :py:class:`Transformer`. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(PipelineStage, self).__init__() - - -@inherit_doc -class Estimator(PipelineStage): - """ - Abstract class for estimators that fit models to data. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(Estimator, self).__init__() - - @abstractmethod - def fit(self, dataset, params={}): - """ - Fits a model to the input dataset with optional parameters. - - :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` - :param params: an optional param map that overwrites embedded - params - :returns: fitted model - """ - raise NotImplementedError() - - -@inherit_doc -class Transformer(PipelineStage): - """ - Abstract class for transformers that transform one dataset into - another. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(Transformer, self).__init__() - - @abstractmethod - def transform(self, dataset, params={}): - """ - Transforms the input dataset with optional parameters. - - :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` - :param params: an optional param map that overwrites embedded - params - :returns: transformed dataset - """ - raise NotImplementedError() - - -@inherit_doc -class Model(Transformer): - """ - Abstract class for models fitted by :py:class:`Estimator`s. - """ - - ___metaclass__ = ABCMeta - - def __init__(self): - super(Model, self).__init__() - - -@inherit_doc -class Pipeline(Estimator): - """ - A simple pipeline, which acts as an estimator. A Pipeline consists - of a sequence of stages, each of which is either an - :py:class:`Estimator` or a :py:class:`Transformer`. When - :py:meth:`Pipeline.fit` is called, the stages are executed in - order. If a stage is an :py:class:`Estimator`, its - :py:meth:`Estimator.fit` method will be called on the input - dataset to fit a model. Then the model, which is a transformer, - will be used to transform the dataset as the input to the next - stage. If a stage is a :py:class:`Transformer`, its - :py:meth:`Transformer.transform` method will be called to produce - the dataset for the next stage. The fitted model from a - :py:class:`Pipeline` is an :py:class:`PipelineModel`, which - consists of fitted models and transformers, corresponding to the - pipeline stages. If there are no stages, the pipeline acts as an - identity transformer. - """ - - def __init__(self): - super(Pipeline, self).__init__() - #: Param for pipeline stages. - self.stages = Param(self, "stages", "pipeline stages") - - def setStages(self, value): - """ - Set pipeline stages. - :param value: a list of transformers or estimators - :return: the pipeline instance - """ - self.paramMap[self.stages] = value - return self - - def getStages(self): - """ - Get pipeline stages. - """ - if self.stages in self.paramMap: - return self.paramMap[self.stages] - - def fit(self, dataset, params={}): - paramMap = self._merge_params(params) - stages = paramMap[self.stages] - for stage in stages: - if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): - raise ValueError( - "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) - indexOfLastEstimator = -1 - for i, stage in enumerate(stages): - if isinstance(stage, Estimator): - indexOfLastEstimator = i - transformers = [] - for i, stage in enumerate(stages): - if i <= indexOfLastEstimator: - if isinstance(stage, Transformer): - transformers.append(stage) - dataset = stage.transform(dataset, paramMap) - else: # must be an Estimator - model = stage.fit(dataset, paramMap) - transformers.append(model) - if i < indexOfLastEstimator: - dataset = model.transform(dataset, paramMap) - else: - transformers.append(stage) - return PipelineModel(transformers) - - -@inherit_doc -class PipelineModel(Model): - """ - Represents a compiled pipeline with transformers and fitted models. - """ - - def __init__(self, transformers): - super(PipelineModel, self).__init__() - self.transformers = transformers - - def transform(self, dataset, params={}): - paramMap = self._merge_params(params) - for t in self.transformers: - dataset = t.transform(dataset, paramMap) - return dataset - - -@inherit_doc -class JavaWrapper(Params): - """ - Utility class to help create wrapper classes from Java/Scala - implementations of pipeline components. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(JavaWrapper, self).__init__() - - @abstractproperty - def _java_class(self): - """ - Fully-qualified class name of the wrapped Java component. - """ - raise NotImplementedError - - def _java_obj(self): - """ - Returns or creates a Java object. - """ - java_obj = _jvm() - for name in self._java_class.split("."): - java_obj = getattr(java_obj, name) - return java_obj() - - def _transfer_params_to_java(self, params, java_obj): - """ - Transforms the embedded params and additional params to the - input Java object. - :param params: additional params (overwriting embedded values) - :param java_obj: Java object to receive the params - """ - paramMap = self._merge_params(params) - for param in self.params: - if param in paramMap: - java_obj.set(param.name, paramMap[param]) - - def _empty_java_param_map(self): - """ - Returns an empty Java ParamMap reference. - """ - return _jvm().org.apache.spark.ml.param.ParamMap() - - def _create_java_param_map(self, params, java_obj): - paramMap = self._empty_java_param_map() - for param, value in params.items(): - if param.parent is self: - paramMap.put(java_obj.getParam(param.name), value) - return paramMap - - -@inherit_doc -class JavaEstimator(Estimator, JavaWrapper): - """ - Base class for :py:class:`Estimator`s that wrap Java/Scala - implementations. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(JavaEstimator, self).__init__() - - @abstractmethod - def _create_model(self, java_model): - """ - Creates a model from the input Java model reference. - """ - raise NotImplementedError - - def _fit_java(self, dataset, params={}): - """ - Fits a Java model to the input dataset. - :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` - :param params: additional params (overwriting embedded values) - :return: fitted Java model - """ - java_obj = self._java_obj() - self._transfer_params_to_java(params, java_obj) - return java_obj.fit(dataset._jschema_rdd, self._empty_java_param_map()) - - def fit(self, dataset, params={}): - java_model = self._fit_java(dataset, params) - return self._create_model(java_model) - - -@inherit_doc -class JavaTransformer(Transformer, JavaWrapper): - """ - Base class for :py:class:`Transformer`s that wrap Java/Scala - implementations. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(JavaTransformer, self).__init__() - - def transform(self, dataset, params={}): - java_obj = self._java_obj() - self._transfer_params_to_java({}, java_obj) - java_param_map = self._create_java_param_map(params, java_obj) - return SchemaRDD(java_obj.transform(dataset._jschema_rdd, java_param_map), - dataset.sql_ctx) - - -@inherit_doc -class JavaModel(JavaTransformer): - """ - Base class for :py:class:`Model`s that wrap Java/Scala - implementations. - """ - - __metaclass__ = ABCMeta - - def __init__(self): - super(JavaTransformer, self).__init__() - - def _java_obj(self): - return self._java_model +__all__ = ["Pipeline", "Transformer", "Estimator"] diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 0384c39b5d5ab..4b8f7fbe7d0ae 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -15,12 +15,14 @@ # limitations under the License. # -from pyspark.sql import inherit_doc -from pyspark.ml import JavaEstimator, JavaModel +from pyspark.ml.util import JavaEstimator, JavaModel, inherit_doc from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ HasRegParam +__all__ = ['LogisticRegression', 'LogisticRegressionModel'] + + @inherit_doc class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter, HasRegParam): @@ -43,32 +45,17 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> print model.transform(test1).first().prediction 1.0 """ - - def __init__(self): - super(LogisticRegression, self).__init__() - - @property - def _java_class(self): - return "org.apache.spark.ml.classification.LogisticRegression" + _java_class = "org.apache.spark.ml.classification.LogisticRegression" def _create_model(self, java_model): return LogisticRegressionModel(java_model) -@inherit_doc class LogisticRegressionModel(JavaModel): """ Model fitted by LogisticRegression. """ - def __init__(self, java_model): - super(LogisticRegressionModel, self).__init__() - self._java_model = java_model - - @property - def _java_class(self): - return "org.apache.spark.ml.classification.LogisticRegressionModel" - if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 45191c8fcb3dc..c5ea41922e1b6 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,11 +15,13 @@ # limitations under the License. # -from pyspark.sql import inherit_doc -from pyspark.ml import JavaTransformer +from pyspark.ml.util import JavaTransformer, inherit_doc from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures +__all__ = ['Tokenizer', 'HashingTF'] + + @inherit_doc class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): """ @@ -36,12 +38,7 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): Row(text=u'a b c', tokens=[u'a', u'b', u'c']) """ - def __init__(self): - super(Tokenizer, self).__init__() - - @property - def _java_class(self): - return "org.apache.spark.ml.feature.Tokenizer" + _java_class = "org.apache.spark.ml.feature.Tokenizer" @inherit_doc @@ -62,12 +59,7 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): (5,[2,3,4],[1.0,1.0,1.0]) """ - def __init__(self): - super(HashingTF, self).__init__() - - @property - def _java_class(self): - return "org.apache.spark.ml.feature.HashingTF" + _java_class = "org.apache.spark.ml.feature.HashingTF" if __name__ == "__main__": diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 80c62c8cdb5d0..2396e20d23fa4 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -19,7 +19,7 @@ from pyspark.ml.util import Identifiable -__all__ = ["Param"] +__all__ = ['Param', 'Params'] class Param(object): diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py new file mode 100644 index 0000000000000..3ef5a201a04b8 --- /dev/null +++ b/python/pyspark/ml/pipeline.py @@ -0,0 +1,153 @@ +# +# 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. +# + +from abc import ABCMeta, abstractmethod + +from pyspark.sql import inherit_doc # TODO: move inherit_doc to Spark Core +from pyspark.ml.param import Param, Params + +__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel'] + + +@inherit_doc +class Estimator(Params): + """ + Abstract class for estimators that fit models to data. + """ + + __metaclass__ = ABCMeta + + @abstractmethod + def fit(self, dataset, params={}): + """ + Fits a model to the input dataset with optional parameters. + + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: an optional param map that overwrites embedded + params + :returns: fitted model + """ + raise NotImplementedError() + + +@inherit_doc +class Transformer(Params): + """ + Abstract class for transformers that transform one dataset into + another. + """ + + __metaclass__ = ABCMeta + + @abstractmethod + def transform(self, dataset, params={}): + """ + Transforms the input dataset with optional parameters. + + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: an optional param map that overwrites embedded + params + :returns: transformed dataset + """ + raise NotImplementedError() + + +@inherit_doc +class Pipeline(Estimator): + """ + A simple pipeline, which acts as an estimator. A Pipeline consists + of a sequence of stages, each of which is either an + :py:class:`Estimator` or a :py:class:`Transformer`. When + :py:meth:`Pipeline.fit` is called, the stages are executed in + order. If a stage is an :py:class:`Estimator`, its + :py:meth:`Estimator.fit` method will be called on the input + dataset to fit a model. Then the model, which is a transformer, + will be used to transform the dataset as the input to the next + stage. If a stage is a :py:class:`Transformer`, its + :py:meth:`Transformer.transform` method will be called to produce + the dataset for the next stage. The fitted model from a + :py:class:`Pipeline` is an :py:class:`PipelineModel`, which + consists of fitted models and transformers, corresponding to the + pipeline stages. If there are no stages, the pipeline acts as an + identity transformer. + """ + + def __init__(self): + super(Pipeline, self).__init__() + #: Param for pipeline stages. + self.stages = Param(self, "stages", "pipeline stages") + + def setStages(self, value): + """ + Set pipeline stages. + :param value: a list of transformers or estimators + :return: the pipeline instance + """ + self.paramMap[self.stages] = value + return self + + def getStages(self): + """ + Get pipeline stages. + """ + if self.stages in self.paramMap: + return self.paramMap[self.stages] + + def fit(self, dataset, params={}): + paramMap = self._merge_params(params) + stages = paramMap[self.stages] + for stage in stages: + if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): + raise ValueError( + "Cannot recognize a pipeline stage of type %s." % type(stage).__name__) + indexOfLastEstimator = -1 + for i, stage in enumerate(stages): + if isinstance(stage, Estimator): + indexOfLastEstimator = i + transformers = [] + for i, stage in enumerate(stages): + if i <= indexOfLastEstimator: + if isinstance(stage, Transformer): + transformers.append(stage) + dataset = stage.transform(dataset, paramMap) + else: # must be an Estimator + model = stage.fit(dataset, paramMap) + transformers.append(model) + if i < indexOfLastEstimator: + dataset = model.transform(dataset, paramMap) + else: + transformers.append(stage) + return PipelineModel(transformers) + + +@inherit_doc +class PipelineModel(Transformer): + """ + Represents a compiled pipeline with transformers and fitted models. + """ + + def __init__(self, transformers): + super(PipelineModel, self).__init__() + self.transformers = transformers + + def transform(self, dataset, params={}): + paramMap = self._merge_params(params) + for t in self.transformers: + dataset = t.transform(dataset, paramMap) + return dataset diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 2f5f2d18d5d01..a154b51c11843 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -32,7 +32,7 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import SchemaRDD -from pyspark.ml import Transformer, Estimator, Model, Pipeline +from pyspark.ml import Transformer, Estimator, Pipeline from pyspark.ml.param import Param @@ -76,7 +76,7 @@ def fit(self, dataset, params={}): return model -class MockModel(MockTransformer, Model): +class MockModel(MockTransformer, Transformer): def __init__(self): super(MockModel, self).__init__() diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 5d74088b0b13e..82e1f9fa087e7 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -16,6 +16,12 @@ # import uuid +from abc import ABCMeta + +from pyspark import SparkContext +from pyspark.sql import DataFrame +from pyspark.ml.param import Params +from pyspark.ml.pipeline import Estimator, Transformer class Identifiable(object): @@ -28,8 +34,144 @@ def __init__(self): #: concatenates the class name, "-", and 8 random hex chars. self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] - def __str__(self): + def __repr__(self): return self.uid - def __repr__(self): - return str(self) + +def inherit_doc(cls): + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls + + +def _jvm(): + """ + Returns the JVM view associated with SparkContext. Must be called + after SparkContext is initialized. + """ + jvm = SparkContext._jvm + if jvm: + return jvm + else: + raise AttributeError("Cannot load _jvm from SparkContext. Is SparkContext initialized?") + + +@inherit_doc +class JavaWrapper(Params): + """ + Utility class to help create wrapper classes from Java/Scala + implementations of pipeline components. + """ + + __metaclass__ = ABCMeta + + #: Fully-qualified class name of the wrapped Java component. + _java_class = None + + def _java_obj(self): + """ + Returns or creates a Java object. + """ + java_obj = _jvm() + for name in self._java_class.split("."): + java_obj = getattr(java_obj, name) + return java_obj() + + def _transfer_params_to_java(self, params, java_obj): + """ + Transforms the embedded params and additional params to the + input Java object. + :param params: additional params (overwriting embedded values) + :param java_obj: Java object to receive the params + """ + paramMap = self._merge_params(params) + for param in self.params: + if param in paramMap: + java_obj.set(param.name, paramMap[param]) + + def _empty_java_param_map(self): + """ + Returns an empty Java ParamMap reference. + """ + return _jvm().org.apache.spark.ml.param.ParamMap() + + def _create_java_param_map(self, params, java_obj): + paramMap = self._empty_java_param_map() + for param, value in params.items(): + if param.parent is self: + paramMap.put(java_obj.getParam(param.name), value) + return paramMap + + +@inherit_doc +class JavaEstimator(Estimator, JavaWrapper): + """ + Base class for :py:class:`Estimator`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def _create_model(self, java_model): + """ + Creates a model from the input Java model reference. + """ + return JavaModel(java_model) + + def _fit_java(self, dataset, params={}): + """ + Fits a Java model to the input dataset. + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: additional params (overwriting embedded values) + :return: fitted Java model + """ + java_obj = self._java_obj() + self._transfer_params_to_java(params, java_obj) + return java_obj.fit(dataset._jschema_rdd, self._empty_java_param_map()) + + def fit(self, dataset, params={}): + java_model = self._fit_java(dataset, params) + return self._create_model(java_model) + + +@inherit_doc +class JavaTransformer(Transformer, JavaWrapper): + """ + Base class for :py:class:`Transformer`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def transform(self, dataset, params={}): + java_obj = self._java_obj() + self._transfer_params_to_java({}, java_obj) + java_param_map = self._create_java_param_map(params, java_obj) + return DataFrame(java_obj.transform(dataset._jschema_rdd, java_param_map), + dataset.sql_ctx) + + +@inherit_doc +class JavaModel(JavaTransformer): + """ + Base class for :py:class:`Model`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def __init__(self, java_model): + super(JavaTransformer, self).__init__() + self._java_model = java_model + + def _java_obj(self): + return self._java_model diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 7d7550c854b2f..c3a6938f56864 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1794,20 +1794,6 @@ def __repr__(self): return "" % ", ".join(self) -def inherit_doc(cls): - for name, func in vars(cls).items(): - # only inherit docstring for public functions - if name.startswith("_"): - continue - if not func.__doc__: - for parent in cls.__bases__: - parent_func = getattr(parent, name, None) - if parent_func and getattr(parent_func, "__doc__", None): - func.__doc__ = parent_func.__doc__ - break - return cls - - class DataFrame(object): """A collection of rows that have the same columns. From 54ca7df537f936eeec00f33a81f08b5148829379 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 27 Jan 2015 22:32:06 -0800 Subject: [PATCH 25/28] fix tests --- python/pyspark/ml/__init__.py | 2 +- python/pyspark/ml/classification.py | 7 +- python/pyspark/ml/feature.py | 12 +-- python/pyspark/ml/param/__init__.py | 17 +++- python/pyspark/ml/pipeline.py | 15 ++- python/pyspark/ml/tests.py | 6 +- python/pyspark/ml/util.py | 147 --------------------------- python/pyspark/ml/wrapper.py | 149 ++++++++++++++++++++++++++++ python/run-tests | 8 ++ 9 files changed, 200 insertions(+), 163 deletions(-) create mode 100644 python/pyspark/ml/wrapper.py diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index 0a065b3155e1a..a4901622bf816 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -18,4 +18,4 @@ from pyspark.ml.param import * from pyspark.ml.pipeline import * -__all__ = ["Pipeline", "Transformer", "Estimator"] +__all__ = ["Param", "Params", "Pipeline", "Transformer", "Estimator"] diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4b8f7fbe7d0ae..6bd2aa8e47837 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -15,7 +15,8 @@ # limitations under the License. # -from pyspark.ml.util import JavaEstimator, JavaModel, inherit_doc +from pyspark.ml.util import inherit_doc +from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import HasFeaturesCol, HasLabelCol, HasPredictionCol, HasMaxIter,\ HasRegParam @@ -39,10 +40,10 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti .setRegParam(0.01) >>> model = lr.fit(dataset) >>> test0 = sqlCtx.inferSchema(sc.parallelize([Row(features=Vectors.dense(-1.0))])) - >>> print model.transform(test0).first().prediction + >>> print model.transform(test0).head().prediction 0.0 >>> test1 = sqlCtx.inferSchema(sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))])) - >>> print model.transform(test1).first().prediction + >>> print model.transform(test1).head().prediction 1.0 """ _java_class = "org.apache.spark.ml.classification.LogisticRegression" diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 2cfe9a8dddc1f..e088acd0ca82d 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,9 +15,9 @@ # limitations under the License. # -from pyspark.ml.util import JavaTransformer, inherit_doc from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures - +from pyspark.ml.util import inherit_doc +from pyspark.ml.wrapper import JavaTransformer __all__ = ['Tokenizer', 'HashingTF'] @@ -33,9 +33,9 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): >>> tokenizer = Tokenizer() \ .setInputCol("text") \ .setOutputCol("words") - >>> print tokenizer.transform(dataset).first() + >>> print tokenizer.transform(dataset).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> print tokenizer.transform(dataset, {tokenizer.outputCol: "tokens"}).first() + >>> print tokenizer.transform(dataset, {tokenizer.outputCol: "tokens"}).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) """ @@ -54,10 +54,10 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): .setNumFeatures(10) \ .setInputCol("words") \ .setOutputCol("features") - >>> print hashingTF.transform(dataset).first().features + >>> print hashingTF.transform(dataset).head().features (10,[7,8,9],[1.0,1.0,1.0]) >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> print hashingTF.transform(dataset, params).first().vector + >>> print hashingTF.transform(dataset, params).head().vector (5,[2,3,4],[1.0,1.0,1.0]) """ diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 2396e20d23fa4..9d657acdd94f4 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -15,13 +15,26 @@ # limitations under the License. # +import uuid from abc import ABCMeta -from pyspark.ml.util import Identifiable - __all__ = ['Param', 'Params'] +class Identifiable(object): + """ + Object with a unique ID. + """ + + def __init__(self): + #: A unique id for the object. The default implementation + #: concatenates the class name, "-", and 8 random hex chars. + self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + + def __repr__(self): + return self.uid + + class Param(object): """ A param with self-contained documentation and optionally default value. diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 3ef5a201a04b8..0c5ec86620a97 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -17,12 +17,25 @@ from abc import ABCMeta, abstractmethod -from pyspark.sql import inherit_doc # TODO: move inherit_doc to Spark Core from pyspark.ml.param import Param, Params __all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel'] +def inherit_doc(cls): + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls + + @inherit_doc class Estimator(Params): """ diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index a154b51c11843..b627c2b4e930b 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -31,12 +31,12 @@ import unittest from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase -from pyspark.sql import SchemaRDD -from pyspark.ml import Transformer, Estimator, Pipeline +from pyspark.sql import DataFrame from pyspark.ml.param import Param +from pyspark.ml.pipeline import Transformer, Estimator, Pipeline -class MockDataset(SchemaRDD): +class MockDataset(DataFrame): def __init__(self): self.index = 0 diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 82e1f9fa087e7..991330f78e983 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -15,28 +15,6 @@ # limitations under the License. # -import uuid -from abc import ABCMeta - -from pyspark import SparkContext -from pyspark.sql import DataFrame -from pyspark.ml.param import Params -from pyspark.ml.pipeline import Estimator, Transformer - - -class Identifiable(object): - """ - Object with a unique ID. - """ - - def __init__(self): - #: A unique id for the object. The default implementation - #: concatenates the class name, "-", and 8 random hex chars. - self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] - - def __repr__(self): - return self.uid - def inherit_doc(cls): for name, func in vars(cls).items(): @@ -50,128 +28,3 @@ def inherit_doc(cls): func.__doc__ = parent_func.__doc__ break return cls - - -def _jvm(): - """ - Returns the JVM view associated with SparkContext. Must be called - after SparkContext is initialized. - """ - jvm = SparkContext._jvm - if jvm: - return jvm - else: - raise AttributeError("Cannot load _jvm from SparkContext. Is SparkContext initialized?") - - -@inherit_doc -class JavaWrapper(Params): - """ - Utility class to help create wrapper classes from Java/Scala - implementations of pipeline components. - """ - - __metaclass__ = ABCMeta - - #: Fully-qualified class name of the wrapped Java component. - _java_class = None - - def _java_obj(self): - """ - Returns or creates a Java object. - """ - java_obj = _jvm() - for name in self._java_class.split("."): - java_obj = getattr(java_obj, name) - return java_obj() - - def _transfer_params_to_java(self, params, java_obj): - """ - Transforms the embedded params and additional params to the - input Java object. - :param params: additional params (overwriting embedded values) - :param java_obj: Java object to receive the params - """ - paramMap = self._merge_params(params) - for param in self.params: - if param in paramMap: - java_obj.set(param.name, paramMap[param]) - - def _empty_java_param_map(self): - """ - Returns an empty Java ParamMap reference. - """ - return _jvm().org.apache.spark.ml.param.ParamMap() - - def _create_java_param_map(self, params, java_obj): - paramMap = self._empty_java_param_map() - for param, value in params.items(): - if param.parent is self: - paramMap.put(java_obj.getParam(param.name), value) - return paramMap - - -@inherit_doc -class JavaEstimator(Estimator, JavaWrapper): - """ - Base class for :py:class:`Estimator`s that wrap Java/Scala - implementations. - """ - - __metaclass__ = ABCMeta - - def _create_model(self, java_model): - """ - Creates a model from the input Java model reference. - """ - return JavaModel(java_model) - - def _fit_java(self, dataset, params={}): - """ - Fits a Java model to the input dataset. - :param dataset: input dataset, which is an instance of - :py:class:`pyspark.sql.SchemaRDD` - :param params: additional params (overwriting embedded values) - :return: fitted Java model - """ - java_obj = self._java_obj() - self._transfer_params_to_java(params, java_obj) - return java_obj.fit(dataset._jschema_rdd, self._empty_java_param_map()) - - def fit(self, dataset, params={}): - java_model = self._fit_java(dataset, params) - return self._create_model(java_model) - - -@inherit_doc -class JavaTransformer(Transformer, JavaWrapper): - """ - Base class for :py:class:`Transformer`s that wrap Java/Scala - implementations. - """ - - __metaclass__ = ABCMeta - - def transform(self, dataset, params={}): - java_obj = self._java_obj() - self._transfer_params_to_java({}, java_obj) - java_param_map = self._create_java_param_map(params, java_obj) - return DataFrame(java_obj.transform(dataset._jschema_rdd, java_param_map), - dataset.sql_ctx) - - -@inherit_doc -class JavaModel(JavaTransformer): - """ - Base class for :py:class:`Model`s that wrap Java/Scala - implementations. - """ - - __metaclass__ = ABCMeta - - def __init__(self, java_model): - super(JavaTransformer, self).__init__() - self._java_model = java_model - - def _java_obj(self): - return self._java_model diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py new file mode 100644 index 0000000000000..9e12ddc3d9b8f --- /dev/null +++ b/python/pyspark/ml/wrapper.py @@ -0,0 +1,149 @@ +# +# 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. +# + +from abc import ABCMeta + +from pyspark import SparkContext +from pyspark.sql import DataFrame +from pyspark.ml.param import Params +from pyspark.ml.pipeline import Estimator, Transformer +from pyspark.ml.util import inherit_doc + + +def _jvm(): + """ + Returns the JVM view associated with SparkContext. Must be called + after SparkContext is initialized. + """ + jvm = SparkContext._jvm + if jvm: + return jvm + else: + raise AttributeError("Cannot load _jvm from SparkContext. Is SparkContext initialized?") + + +@inherit_doc +class JavaWrapper(Params): + """ + Utility class to help create wrapper classes from Java/Scala + implementations of pipeline components. + """ + + __metaclass__ = ABCMeta + + #: Fully-qualified class name of the wrapped Java component. + _java_class = None + + def _java_obj(self): + """ + Returns or creates a Java object. + """ + java_obj = _jvm() + for name in self._java_class.split("."): + java_obj = getattr(java_obj, name) + return java_obj() + + def _transfer_params_to_java(self, params, java_obj): + """ + Transforms the embedded params and additional params to the + input Java object. + :param params: additional params (overwriting embedded values) + :param java_obj: Java object to receive the params + """ + paramMap = self._merge_params(params) + for param in self.params: + if param in paramMap: + java_obj.set(param.name, paramMap[param]) + + def _empty_java_param_map(self): + """ + Returns an empty Java ParamMap reference. + """ + return _jvm().org.apache.spark.ml.param.ParamMap() + + def _create_java_param_map(self, params, java_obj): + paramMap = self._empty_java_param_map() + for param, value in params.items(): + if param.parent is self: + paramMap.put(java_obj.getParam(param.name), value) + return paramMap + + +@inherit_doc +class JavaEstimator(Estimator, JavaWrapper): + """ + Base class for :py:class:`Estimator`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def _create_model(self, java_model): + """ + Creates a model from the input Java model reference. + """ + return JavaModel(java_model) + + def _fit_java(self, dataset, params={}): + """ + Fits a Java model to the input dataset. + :param dataset: input dataset, which is an instance of + :py:class:`pyspark.sql.SchemaRDD` + :param params: additional params (overwriting embedded values) + :return: fitted Java model + """ + java_obj = self._java_obj() + self._transfer_params_to_java(params, java_obj) + return java_obj.fit(dataset._jdf, self._empty_java_param_map()) + + def fit(self, dataset, params={}): + java_model = self._fit_java(dataset, params) + return self._create_model(java_model) + + +@inherit_doc +class JavaTransformer(Transformer, JavaWrapper): + """ + Base class for :py:class:`Transformer`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def transform(self, dataset, params={}): + java_obj = self._java_obj() + self._transfer_params_to_java({}, java_obj) + java_param_map = self._create_java_param_map(params, java_obj) + return DataFrame(java_obj.transform(dataset._jdf, java_param_map), + dataset.sql_ctx) + + +@inherit_doc +class JavaModel(JavaTransformer): + """ + Base class for :py:class:`Model`s that wrap Java/Scala + implementations. + """ + + __metaclass__ = ABCMeta + + def __init__(self, java_model): + super(JavaTransformer, self).__init__() + self._java_model = java_model + + def _java_obj(self): + return self._java_model diff --git a/python/run-tests b/python/run-tests index 9ee19ed6e6b26..57e58c1341c62 100755 --- a/python/run-tests +++ b/python/run-tests @@ -81,6 +81,13 @@ function run_mllib_tests() { run_test "pyspark/mllib/tests.py" } +function run_ml_tests() { + echo "Run ml tests ..." + run_test "pyspark/ml/feature.py" + run_test "pyspark/ml/classification.py" + run_test "pyspark/ml/tests.py" +} + function run_streaming_tests() { echo "Run streaming tests ..." run_test "pyspark/streaming/util.py" @@ -102,6 +109,7 @@ $PYSPARK_PYTHON --version run_core_tests run_sql_tests run_mllib_tests +run_ml_tests run_streaming_tests # Try to test with PyPy From 14ae7e2df5c0d174869efae0715d8156a74c9c2b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 27 Jan 2015 22:44:13 -0800 Subject: [PATCH 26/28] fix docs --- python/docs/conf.py | 4 ++-- python/docs/index.rst | 1 + python/docs/pyspark.ml.rst | 12 ------------ python/pyspark/ml/__init__.py | 2 +- 4 files changed, 4 insertions(+), 15 deletions(-) diff --git a/python/docs/conf.py b/python/docs/conf.py index e58d97ae6a746..b00dce95d65b4 100644 --- a/python/docs/conf.py +++ b/python/docs/conf.py @@ -55,9 +55,9 @@ # built documents. # # The short X.Y version. -version = '1.2-SNAPSHOT' +version = '1.3-SNAPSHOT' # The full version, including alpha/beta/rc tags. -release = '1.2-SNAPSHOT' +release = '1.3-SNAPSHOT' # The language for content autogenerated by Sphinx. Refer to documentation # for a list of supported languages. diff --git a/python/docs/index.rst b/python/docs/index.rst index 703bef644de28..d150de9d5c502 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -14,6 +14,7 @@ Contents: pyspark pyspark.sql pyspark.streaming + pyspark.ml pyspark.mllib diff --git a/python/docs/pyspark.ml.rst b/python/docs/pyspark.ml.rst index 064fcd9c03204..f10d1339a9a8f 100644 --- a/python/docs/pyspark.ml.rst +++ b/python/docs/pyspark.ml.rst @@ -10,16 +10,6 @@ pyspark.ml module .. automodule:: pyspark.ml :members: :undoc-members: - :show-inheritance: - :inherited-members: - -pyspark.ml.param module ------------------------ - -.. automodule:: pyspark.ml.param - :members: - :undoc-members: - :show-inheritance: :inherited-members: pyspark.ml.feature module @@ -28,7 +18,6 @@ pyspark.ml.feature module .. automodule:: pyspark.ml.feature :members: :undoc-members: - :show-inheritance: :inherited-members: pyspark.ml.classification module @@ -37,5 +26,4 @@ pyspark.ml.classification module .. automodule:: pyspark.ml.classification :members: :undoc-members: - :show-inheritance: :inherited-members: diff --git a/python/pyspark/ml/__init__.py b/python/pyspark/ml/__init__.py index a4901622bf816..47fed80f42e13 100644 --- a/python/pyspark/ml/__init__.py +++ b/python/pyspark/ml/__init__.py @@ -18,4 +18,4 @@ from pyspark.ml.param import * from pyspark.ml.pipeline import * -__all__ = ["Param", "Params", "Pipeline", "Transformer", "Estimator"] +__all__ = ["Param", "Params", "Transformer", "Estimator", "Pipeline"] From edbd6feb8a8dc79dcd8b5185e5ea48f332bbfae2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 28 Jan 2015 12:41:41 -0800 Subject: [PATCH 27/28] move Identifiable to ml.util --- python/pyspark/ml/param/__init__.py | 16 ++-------------- python/pyspark/ml/util.py | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 14 deletions(-) diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 9d657acdd94f4..5566792cead48 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -15,24 +15,12 @@ # limitations under the License. # -import uuid from abc import ABCMeta -__all__ = ['Param', 'Params'] - - -class Identifiable(object): - """ - Object with a unique ID. - """ +from pyspark.ml.util import Identifiable - def __init__(self): - #: A unique id for the object. The default implementation - #: concatenates the class name, "-", and 8 random hex chars. - self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] - def __repr__(self): - return self.uid +__all__ = ['Param', 'Params'] class Param(object): diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 991330f78e983..b1caa84b6306a 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -15,6 +15,8 @@ # limitations under the License. # +import uuid + def inherit_doc(cls): for name, func in vars(cls).items(): @@ -28,3 +30,17 @@ def inherit_doc(cls): func.__doc__ = parent_func.__doc__ break return cls + + +class Identifiable(object): + """ + Object with a unique ID. + """ + + def __init__(self): + #: A unique id for the object. The default implementation + #: concatenates the class name, "-", and 8 random hex chars. + self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + + def __repr__(self): + return self.uid From 415268e19ebf9d48cb4a50773f446ebbe8a902cd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 28 Jan 2015 15:53:38 -0800 Subject: [PATCH 28/28] remove inherit_doc from __init__ --- python/pyspark/ml/pipeline.py | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 0c5ec86620a97..2d239f8c802a0 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -18,22 +18,10 @@ from abc import ABCMeta, abstractmethod from pyspark.ml.param import Param, Params - -__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel'] +from pyspark.ml.util import inherit_doc -def inherit_doc(cls): - for name, func in vars(cls).items(): - # only inherit docstring for public functions - if name.startswith("_"): - continue - if not func.__doc__: - for parent in cls.__bases__: - parent_func = getattr(parent, name, None) - if parent_func and getattr(parent_func, "__doc__", None): - func.__doc__ = parent_func.__doc__ - break - return cls +__all__ = ['Estimator', 'Transformer', 'Pipeline', 'PipelineModel'] @inherit_doc