spark-instrumented-optimizer/python/pyspark/ml/wrapper.py
Bryan Cutler 828213d4ca [SPARK-13937][PYSPARK][ML] Change JavaWrapper _java_obj from static to member variable
## What changes were proposed in this pull request?
In PySpark wrapper.py JavaWrapper change _java_obj from an unused static variable to a member variable that is consistent with usage in derived classes.

## How was this patch tested?
Ran python tests for ML and MLlib.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11767 from BryanCutler/JavaWrapper-static-_java_obj-SPARK-13937.
2016-03-17 10:16:51 -07:00

226 lines
7.8 KiB
Python

#
# 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.sql import DataFrame
from pyspark.ml import Estimator, Transformer, Model
from pyspark.ml.param import Params
from pyspark.ml.util import _jvm
from pyspark.mllib.common import inherit_doc, _java2py, _py2java
@inherit_doc
class JavaWrapper(Params):
"""
Utility class to help create wrapper classes from Java/Scala
implementations of pipeline components.
"""
__metaclass__ = ABCMeta
def __init__(self):
"""
Initialize the wrapped java object to None
"""
super(JavaWrapper, self).__init__()
#: The wrapped Java companion object. Subclasses should initialize
#: it properly. The param values in the Java object should be
#: synced with the Python wrapper in fit/transform/evaluate/copy.
self._java_obj = None
@staticmethod
def _new_java_obj(java_class, *args):
"""
Construct a new Java object.
"""
sc = SparkContext._active_spark_context
java_obj = _jvm()
for name in java_class.split("."):
java_obj = getattr(java_obj, name)
java_args = [_py2java(sc, arg) for arg in args]
return java_obj(*java_args)
def _make_java_param_pair(self, param, value):
"""
Makes a Java parm pair.
"""
sc = SparkContext._active_spark_context
param = self._resolveParam(param)
java_param = self._java_obj.getParam(param.name)
java_value = _py2java(sc, value)
return java_param.w(java_value)
def _transfer_params_to_java(self):
"""
Transforms the embedded params to the companion Java object.
"""
paramMap = self.extractParamMap()
for param in self.params:
if param in paramMap:
pair = self._make_java_param_pair(param, paramMap[param])
self._java_obj.set(pair)
def _transfer_params_from_java(self):
"""
Transforms the embedded params from the companion Java object.
"""
sc = SparkContext._active_spark_context
for param in self.params:
if self._java_obj.hasParam(param.name):
java_param = self._java_obj.getParam(param.name)
if self._java_obj.isDefined(java_param):
value = _java2py(sc, self._java_obj.getOrDefault(java_param))
self._paramMap[param] = value
@staticmethod
def _empty_java_param_map():
"""
Returns an empty Java ParamMap reference.
"""
return _jvm().org.apache.spark.ml.param.ParamMap()
def _transfer_stage_to_java(self):
self._transfer_params_to_java()
return self._java_obj
@staticmethod
def _transfer_stage_from_java(java_stage):
def __get_class(clazz):
"""
Loads Python class from its name.
"""
parts = clazz.split('.')
module = ".".join(parts[:-1])
m = __import__(module)
for comp in parts[1:]:
m = getattr(m, comp)
return m
stage_name = java_stage.getClass().getName().replace("org.apache.spark", "pyspark")
# Generate a default new instance from the stage_name class.
py_stage = __get_class(stage_name)()
assert(isinstance(py_stage, JavaWrapper),
"Python side implementation is not supported in the meta-PipelineStage currently.")
# Load information from java_stage to the instance.
py_stage._java_obj = java_stage
py_stage._resetUid(java_stage.uid())
py_stage._transfer_params_from_java()
return py_stage
@inherit_doc
class JavaEstimator(Estimator, JavaWrapper):
"""
Base class for :py:class:`Estimator`s that wrap Java/Scala
implementations.
"""
__metaclass__ = ABCMeta
@abstractmethod
def _create_model(self, java_model):
"""
Creates a model from the input Java model reference.
"""
raise NotImplementedError()
def _fit_java(self, dataset):
"""
Fits a Java model to the input dataset.
:param dataset: input dataset, which is an instance of
:py:class:`pyspark.sql.DataFrame`
:param params: additional params (overwriting embedded values)
:return: fitted Java model
"""
self._transfer_params_to_java()
return self._java_obj.fit(dataset._jdf)
def _fit(self, dataset):
java_model = self._fit_java(dataset)
return self._create_model(java_model)
@inherit_doc
class JavaTransformer(Transformer, JavaWrapper):
"""
Base class for :py:class:`Transformer`s that wrap Java/Scala
implementations. Subclasses should ensure they have the transformer Java object
available as _java_obj.
"""
__metaclass__ = ABCMeta
def _transform(self, dataset):
self._transfer_params_to_java()
return DataFrame(self._java_obj.transform(dataset._jdf), dataset.sql_ctx)
@inherit_doc
class JavaModel(Model, JavaTransformer):
"""
Base class for :py:class:`Model`s that wrap Java/Scala
implementations. Subclasses should inherit this class before
param mix-ins, because this sets the UID from the Java model.
"""
__metaclass__ = ABCMeta
def __init__(self, java_model=None):
"""
Initialize this instance with a Java model object.
Subclasses should call this constructor, initialize params,
and then call _transformer_params_from_java.
This instance can be instantiated without specifying java_model,
it will be assigned after that, but this scenario only used by
:py:class:`JavaMLReader` to load models. This is a bit of a
hack, but it is easiest since a proper fix would require
MLReader (in pyspark.ml.util) to depend on these wrappers, but
these wrappers depend on pyspark.ml.util (both directly and via
other ML classes).
"""
super(JavaModel, self).__init__()
if java_model is not None:
self._java_obj = java_model
self.uid = java_model.uid()
def copy(self, extra=None):
"""
Creates a copy of this instance with the same uid and some
extra params. This implementation first calls Params.copy and
then make a copy of the companion Java model with extra params.
So both the Python wrapper and the Java model get copied.
:param extra: Extra parameters to copy to the new instance
:return: Copy of this instance
"""
if extra is None:
extra = dict()
that = super(JavaModel, self).copy(extra)
if self._java_obj is not None:
that._java_obj = self._java_obj.copy(self._empty_java_param_map())
that._transfer_params_to_java()
return that
def _call_java(self, name, *args):
m = getattr(self._java_obj, name)
sc = SparkContext._active_spark_context
java_args = [_py2java(sc, arg) for arg in args]
return _java2py(sc, m(*java_args))