You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by huaxingao <gi...@git.apache.org> on 2018/04/21 04:22:08 UTC

[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

GitHub user huaxingao opened a pull request:

    https://github.com/apache/spark/pull/21119

    [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

    ## What changes were proposed in this pull request?
    
    add spark.ml Python API for PIC
    
    ## How was this patch tested?
    
    add doctest

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/huaxingao/spark spark_19826

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/21119.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #21119
    
----
commit 53d7763b58d05a6baf9fcf1cef2ae327a5d42e04
Author: Huaxin Gao <hu...@...>
Date:   2018-04-21T04:15:37Z

    [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184838934
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,204 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +@inherit_doc
    +class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,
    +                               JavaMLReadable, JavaMLWritable):
    +    """
    +    .. note:: Experimental
    +    Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
    +    <a href=http://www.icml2010.org/papers/387.pdf>Lin and Cohen</a>. From the abstract:
    +    PIC finds a very low-dimensional embedding of a dataset using truncated power
    +    iteration on a normalized pair-wise similarity matrix of the data.
    +
    +    PIC takes an affinity matrix between items (or vertices) as input.  An affinity matrix
    +    is a symmetric matrix whose entries are non-negative similarities between items.
    +    PIC takes this matrix (or graph) as an adjacency matrix.  Specifically, each input row
    +    includes:
    +
    +     - :py:class:`idCol`: vertex ID
    +     - :py:class:`neighborsCol`: neighbors of vertex in :py:class:`idCol`
    +     - :py:class:`similaritiesCol`: non-negative weights (similarities) of edges between the
    +        vertex in :py:class:`idCol` and each neighbor in :py:class:`neighborsCol`
    +
    +    PIC returns a cluster assignment for each input vertex.  It appends a new column
    +    :py:class:`predictionCol` containing the cluster assignment in :py:class:`[0,k)` for
    +    each row (vertex).
    +
    +    Notes:
    +
    +     - [[PowerIterationClustering]] is a transformer with an expensive [[transform]] operation.
    +        Transform runs the iterative PIC algorithm to cluster the whole input dataset.
    +     - Input validation: This validates that similarities are non-negative but does NOT validate
    +        that the input matrix is symmetric.
    +
    +    @see <a href=http://en.wikipedia.org/wiki/Spectral_clustering>
    +    Spectral clustering (Wikipedia)</a>
    +
    +    >>> from pyspark.sql.types import ArrayType, DoubleType, LongType, StructField, StructType
    +    >>> similarities = [((long)(1), [0], [0.5]), ((long)(2), [0, 1], [0.7,0.5]), \
    +                        ((long)(3), [0, 1, 2], [0.9, 0.7, 0.5]), \
    +                        ((long)(4), [0, 1, 2, 3], [1.1, 0.9, 0.7,0.5]), \
    +                        ((long)(5), [0, 1, 2, 3, 4], [1.3, 1.1, 0.9, 0.7,0.5])]
    +    >>> rdd = sc.parallelize(similarities, 2)
    +    >>> schema = StructType([StructField("id", LongType(), False), \
    +                 StructField("neighbors", ArrayType(LongType(), False), True), \
    +                 StructField("similarities", ArrayType(DoubleType(), False), True)])
    +    >>> df = spark.createDataFrame(rdd, schema)
    +    >>> pic = PowerIterationClustering()
    +    >>> result = pic.setK(2).setMaxIter(10).transform(df)
    +    >>> predictions = sorted(set([(i[0], i[1]) for i in result.select(result.id, result.prediction)
    +    ...     .collect()]), key=lambda x: x[0])
    +    >>> predictions[0]
    +    (1, 1)
    +    >>> predictions[1]
    +    (2, 1)
    +    >>> predictions[2]
    +    (3, 0)
    +    >>> predictions[3]
    +    (4, 0)
    +    >>> predictions[4]
    +    (5, 0)
    +    >>> pic_path = temp_path + "/pic"
    +    >>> pic.save(pic_path)
    +    >>> pic2 = PowerIterationClustering.load(pic_path)
    +    >>> pic2.getK()
    +    2
    +    >>> pic2.getMaxIter()
    +    10
    +    >>> pic3 = PowerIterationClustering(k=4, initMode="degree")
    +    >>> pic3.getIdCol()
    +    'id'
    +    >>> pic3.getK()
    +    4
    +    >>> pic3.getMaxIter()
    +    20
    +    >>> pic3.getInitMode()
    +    'degree'
    +
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @keyword_only
    +    def __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities")
    +        """
    +        super(PowerIterationClustering, self).__init__()
    +        self._java_obj = self._new_java_obj(
    +            "org.apache.spark.ml.clustering.PowerIterationClustering", self.uid)
    +        self._setDefault(k=2, maxIter=20, initMode="random", idCol="id", neighborsCol="neighbors",
    +                         similaritiesCol="similarities")
    +        kwargs = self._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    @keyword_only
    +    @since("2.4.0")
    +    def setParams(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                  idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        setParams(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                  idCol="id", neighborsCol="neighbors", similaritiesCol="similarities")
    +        Sets params for PowerIterationClustering.
    +        """
    +        kwargs = self._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("2.4.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +        """
    +        return self._set(k=value)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of :py:attr:`k`.
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def setInitMode(self, value):
    +        """
    +        Sets the value of :py:attr:`initMode`.
    +        """
    +        return self._set(initMode=value)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def setIdCol(self, value):
    +        """
    +        Sets the value of :py:attr:`idCol`.
    +        """
    +        return self._set(idCol=value)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of :py:attr:`idCol`.
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def setNeighborsCol(self, value):
    +        """
    +        Sets the value of :py:attr:`neighborsCol.
    --- End diff --
    
    Missing the left back quote.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @huaxingao We updated the Scala/Java API in https://github.com/apache/spark/pull/21493. Could you update this PR for the Python API? It should be similar to the PrefixSpan Python API (https://github.com/apache/spark/commit/90ae98d1accb3e4b7d381de072257bdece8dd7e0), which is neither a transformer nor an estimator. Let me know if you don't have time. @WeichenXu123  could update the Python API as well.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @huaxingao Create a new PR is better I think.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89965/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2598/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89672 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89672/testReport)** for PR 21119 at commit [`53d7763`](https://github.com/apache/spark/commit/53d7763b58d05a6baf9fcf1cef2ae327a5d42e04).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):`
      * `class PowerIterationClustering(JavaTransformer, _PowerIterationClusteringParams, JavaMLReadable,`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @jkbradley 
    Thanks for letting me know. I will change the python API accordingly after the new scala version is in. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2731/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184838848
  
    --- Diff: mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala ---
    @@ -97,13 +97,15 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has
       def getNeighborsCol: String = $(neighborsCol)
     
       /**
    -   * Param for the name of the input column for neighbors in the adjacency list representation.
    +   * Param for the name of the input column for non-negative weights (similarities) of edges
    +   * between the vertex in `idCol` and each neighbor in `neighborsCol`.
    --- End diff --
    
    Good catch!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @huaxingao Any updates?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2730/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184342231
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    --- End diff --
    
    Why not directly add params into class `PowerIterationClustering`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184345688
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    +    """
    +    Params for :py:attr:`PowerIterationClustering`.
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    --- End diff --
    
    Should use:
    :py:attr:`k`
    and update everywhere else.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89965 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89965/testReport)** for PR 21119 at commit [`c25d3dc`](https://github.com/apache/spark/commit/c25d3dcb11eff13bfe1092e1dc64c035335b852b).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @mengxr @WeichenXu123 I will close this one and submit a new PR soon. Thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89737/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89946/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184808830
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    --- End diff --
    
    @WeichenXu123 Thanks for your review. The params can be either inside class PowerIterationClustering or separate. I will move them back inside class PowerIterationClustering, to be consistent with the params in the other classes in clustering. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2742/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89737 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89737/testReport)** for PR 21119 at commit [`387d6ff`](https://github.com/apache/spark/commit/387d6ffa8234f365e0de250082707187f67ecf61).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184346287
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    +    """
    +    Params for :py:attr:`PowerIterationClustering`.
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of `idCol`
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def getNeighborsCol(self):
    +        """
    +        Gets the value of `neighborsCol`
    +        """
    +        return self.getOrDefault(self.neighborsCol)
    +
    +    @since("2.4.0")
    +    def getSimilaritiesCol(self):
    +        """
    +        Gets the value of `similaritiesCol`
    +        """
    +        return self.getOrDefault(self.binary)
    +
    +
    +@inherit_doc
    +class PowerIterationClustering(JavaTransformer, _PowerIterationClusteringParams, JavaMLReadable,
    +                               JavaMLWritable):
    +    """
    +    Model produced by [[PowerIterationClustering]].
    +    >>> from pyspark.sql.types import ArrayType, DoubleType, LongType, StructField, StructType
    +    >>> import math
    +    >>> def genCircle(r, n):
    +    ...     points = []
    +    ...     for i in range(0, n):
    +    ...         theta = 2.0 * math.pi * i / n
    +    ...         points.append((r * math.cos(theta), r * math.sin(theta)))
    +    ...     return points
    +    >>> def sim(x, y):
    +    ...     dist = (x[0] - y[0]) * (x[0] - y[0]) + (x[1] - y[1]) * (x[1] - y[1])
    +    ...     return math.exp(-dist / 2.0)
    +    >>> r1 = 1.0
    +    >>> n1 = 10
    +    >>> r2 = 4.0
    +    >>> n2 = 40
    +    >>> n = n1 + n2
    +    >>> points = genCircle(r1, n1) + genCircle(r2, n2)
    +    >>> similarities = []
    +    >>> for i in range (1, n):
    +    ...    neighbor = []
    +    ...    weight = []
    +    ...    for j in range (i):
    +    ...        neighbor.append((long)(j))
    +    ...        weight.append(sim(points[i], points[j]))
    +    ...    similarities.append([(long)(i), neighbor, weight])
    --- End diff --
    
    The doctest code looks like too long, maybe more proper to put it in examples.
    Could you replace the data generation code here by using a simple hardcoded dataset ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89965 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89965/testReport)** for PR 21119 at commit [`c25d3dc`](https://github.com/apache/spark/commit/c25d3dcb11eff13bfe1092e1dc64c035335b852b).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89943 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89943/testReport)** for PR 21119 at commit [`6d00f34`](https://github.com/apache/spark/commit/6d00f343f5c78fbe290793fe85cbc3deed53cf3e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89672 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89672/testReport)** for PR 21119 at commit [`53d7763`](https://github.com/apache/spark/commit/53d7763b58d05a6baf9fcf1cef2ae327a5d42e04).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @jkbradley Could you please review when you have time? Thank you very much in advance!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184838981
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,204 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +@inherit_doc
    +class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,
    +                               JavaMLReadable, JavaMLWritable):
    +    """
    +    .. note:: Experimental
    +    Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
    +    <a href=http://www.icml2010.org/papers/387.pdf>Lin and Cohen</a>. From the abstract:
    +    PIC finds a very low-dimensional embedding of a dataset using truncated power
    +    iteration on a normalized pair-wise similarity matrix of the data.
    +
    +    PIC takes an affinity matrix between items (or vertices) as input.  An affinity matrix
    +    is a symmetric matrix whose entries are non-negative similarities between items.
    +    PIC takes this matrix (or graph) as an adjacency matrix.  Specifically, each input row
    +    includes:
    +
    +     - :py:class:`idCol`: vertex ID
    +     - :py:class:`neighborsCol`: neighbors of vertex in :py:class:`idCol`
    +     - :py:class:`similaritiesCol`: non-negative weights (similarities) of edges between the
    +        vertex in :py:class:`idCol` and each neighbor in :py:class:`neighborsCol`
    +
    +    PIC returns a cluster assignment for each input vertex.  It appends a new column
    +    :py:class:`predictionCol` containing the cluster assignment in :py:class:`[0,k)` for
    +    each row (vertex).
    +
    +    Notes:
    +
    +     - [[PowerIterationClustering]] is a transformer with an expensive [[transform]] operation.
    +        Transform runs the iterative PIC algorithm to cluster the whole input dataset.
    +     - Input validation: This validates that similarities are non-negative but does NOT validate
    +        that the input matrix is symmetric.
    +
    +    @see <a href=http://en.wikipedia.org/wiki/Spectral_clustering>
    +    Spectral clustering (Wikipedia)</a>
    +
    +    >>> from pyspark.sql.types import ArrayType, DoubleType, LongType, StructField, StructType
    +    >>> similarities = [((long)(1), [0], [0.5]), ((long)(2), [0, 1], [0.7,0.5]), \
    +                        ((long)(3), [0, 1, 2], [0.9, 0.7, 0.5]), \
    +                        ((long)(4), [0, 1, 2, 3], [1.1, 0.9, 0.7,0.5]), \
    +                        ((long)(5), [0, 1, 2, 3, 4], [1.3, 1.1, 0.9, 0.7,0.5])]
    +    >>> rdd = sc.parallelize(similarities, 2)
    +    >>> schema = StructType([StructField("id", LongType(), False), \
    +                 StructField("neighbors", ArrayType(LongType(), False), True), \
    +                 StructField("similarities", ArrayType(DoubleType(), False), True)])
    +    >>> df = spark.createDataFrame(rdd, schema)
    +    >>> pic = PowerIterationClustering()
    +    >>> result = pic.setK(2).setMaxIter(10).transform(df)
    +    >>> predictions = sorted(set([(i[0], i[1]) for i in result.select(result.id, result.prediction)
    +    ...     .collect()]), key=lambda x: x[0])
    +    >>> predictions[0]
    +    (1, 1)
    +    >>> predictions[1]
    +    (2, 1)
    +    >>> predictions[2]
    +    (3, 0)
    +    >>> predictions[3]
    +    (4, 0)
    +    >>> predictions[4]
    +    (5, 0)
    +    >>> pic_path = temp_path + "/pic"
    +    >>> pic.save(pic_path)
    +    >>> pic2 = PowerIterationClustering.load(pic_path)
    +    >>> pic2.getK()
    +    2
    +    >>> pic2.getMaxIter()
    +    10
    +    >>> pic3 = PowerIterationClustering(k=4, initMode="degree")
    +    >>> pic3.getIdCol()
    +    'id'
    +    >>> pic3.getK()
    +    4
    +    >>> pic3.getMaxIter()
    +    20
    +    >>> pic3.getInitMode()
    +    'degree'
    +
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @keyword_only
    +    def __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities")
    +        """
    +        super(PowerIterationClustering, self).__init__()
    +        self._java_obj = self._new_java_obj(
    +            "org.apache.spark.ml.clustering.PowerIterationClustering", self.uid)
    +        self._setDefault(k=2, maxIter=20, initMode="random", idCol="id", neighborsCol="neighbors",
    +                         similaritiesCol="similarities")
    +        kwargs = self._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    @keyword_only
    +    @since("2.4.0")
    +    def setParams(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                  idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        setParams(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                  idCol="id", neighborsCol="neighbors", similaritiesCol="similarities")
    +        Sets params for PowerIterationClustering.
    +        """
    +        kwargs = self._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("2.4.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +        """
    +        return self._set(k=value)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of :py:attr:`k`.
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def setInitMode(self, value):
    +        """
    +        Sets the value of :py:attr:`initMode`.
    +        """
    +        return self._set(initMode=value)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def setIdCol(self, value):
    +        """
    +        Sets the value of :py:attr:`idCol`.
    +        """
    +        return self._set(idCol=value)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of :py:attr:`idCol`.
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def setNeighborsCol(self, value):
    +        """
    +        Sets the value of :py:attr:`neighborsCol.
    +        """
    +        return self._set(neighborsCol=value)
    +
    +    @since("2.4.0")
    +    def getNeighborsCol(self):
    +        """
    +        Gets the value of :py:attr:`neighborsCol`.
    +        """
    +        return self.getOrDefault(self.neighborsCol)
    +
    +    @since("2.4.0")
    +    def setSimilaritiesCol(self, value):
    +        """
    +        Sets the value of :py:attr:`similaritiesCol`.
    +        """
    +        return self._set(similaritiesCol=value)
    +
    +    @since("2.4.0")
    +    def getSimilaritiesCol(self):
    +        """
    +        Gets the value of :py:attr:`similaritiesCol`.
    +        """
    +        return self.getOrDefault(self.binary)
    --- End diff --
    
    `self.binary` -> `self.similaritiesCol`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89946 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89946/testReport)** for PR 21119 at commit [`a6b1822`](https://github.com/apache/spark/commit/a6b18222b65e878e22ddf8f2d340aa3127c99e0c).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2559/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89970 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89970/testReport)** for PR 21119 at commit [`ae9f953`](https://github.com/apache/spark/commit/ae9f953d4a06228b6bf7b6867f031a1bfc84d1e2).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89943/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184343934
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    +    """
    +    Params for :py:attr:`PowerIterationClustering`.
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of `idCol`
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def getNeighborsCol(self):
    +        """
    +        Gets the value of `neighborsCol`
    +        """
    +        return self.getOrDefault(self.neighborsCol)
    +
    +    @since("2.4.0")
    +    def getSimilaritiesCol(self):
    +        """
    +        Gets the value of `similaritiesCol`
    +        """
    +        return self.getOrDefault(self.binary)
    +
    +
    +@inherit_doc
    +class PowerIterationClustering(JavaTransformer, _PowerIterationClusteringParams, JavaMLReadable,
    +                               JavaMLWritable):
    +    """
    +    Model produced by [[PowerIterationClustering]].
    --- End diff --
    
    The doc is wrong. Copy doc from scala side.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @mengxr Sorry for the delay. I will submit an update later today. Do you want me to close this PR and do a new one? or just update this PR? 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2746/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184839152
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,204 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +@inherit_doc
    +class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,
    +                               JavaMLReadable, JavaMLWritable):
    +    """
    +    .. note:: Experimental
    +    Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
    +    <a href=http://www.icml2010.org/papers/387.pdf>Lin and Cohen</a>. From the abstract:
    +    PIC finds a very low-dimensional embedding of a dataset using truncated power
    +    iteration on a normalized pair-wise similarity matrix of the data.
    +
    +    PIC takes an affinity matrix between items (or vertices) as input.  An affinity matrix
    +    is a symmetric matrix whose entries are non-negative similarities between items.
    +    PIC takes this matrix (or graph) as an adjacency matrix.  Specifically, each input row
    +    includes:
    +
    +     - :py:class:`idCol`: vertex ID
    +     - :py:class:`neighborsCol`: neighbors of vertex in :py:class:`idCol`
    +     - :py:class:`similaritiesCol`: non-negative weights (similarities) of edges between the
    +        vertex in :py:class:`idCol` and each neighbor in :py:class:`neighborsCol`
    +
    +    PIC returns a cluster assignment for each input vertex.  It appends a new column
    +    :py:class:`predictionCol` containing the cluster assignment in :py:class:`[0,k)` for
    +    each row (vertex).
    +
    +    Notes:
    +
    +     - [[PowerIterationClustering]] is a transformer with an expensive [[transform]] operation.
    +        Transform runs the iterative PIC algorithm to cluster the whole input dataset.
    +     - Input validation: This validates that similarities are non-negative but does NOT validate
    +        that the input matrix is symmetric.
    +
    +    @see <a href=http://en.wikipedia.org/wiki/Spectral_clustering>
    --- End diff --
    
    Use `.. seealso::`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by jkbradley <gi...@git.apache.org>.
Github user jkbradley commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    I think we messed up with the original PIC API.  Could you please check out my comment here https://issues.apache.org/jira/browse/SPARK-15784 ?  If others agree, I'll revert the Scala API and we can work on adding a modified version.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89735/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184344777
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    +    """
    +    Params for :py:attr:`PowerIterationClustering`.
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of `idCol`
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def getNeighborsCol(self):
    +        """
    +        Gets the value of `neighborsCol`
    +        """
    +        return self.getOrDefault(self.neighborsCol)
    +
    +    @since("2.4.0")
    +    def getSimilaritiesCol(self):
    +        """
    +        Gets the value of `similaritiesCol`
    +        """
    +        return self.getOrDefault(self.binary)
    +
    +
    +@inherit_doc
    +class PowerIterationClustering(JavaTransformer, _PowerIterationClusteringParams, JavaMLReadable,
    +                               JavaMLWritable):
    +    """
    +    Model produced by [[PowerIterationClustering]].
    +    >>> from pyspark.sql.types import ArrayType, DoubleType, LongType, StructField, StructType
    +    >>> import math
    +    >>> def genCircle(r, n):
    +    ...     points = []
    +    ...     for i in range(0, n):
    +    ...         theta = 2.0 * math.pi * i / n
    +    ...         points.append((r * math.cos(theta), r * math.sin(theta)))
    +    ...     return points
    +    >>> def sim(x, y):
    +    ...     dist = (x[0] - y[0]) * (x[0] - y[0]) + (x[1] - y[1]) * (x[1] - y[1])
    +    ...     return math.exp(-dist / 2.0)
    +    >>> r1 = 1.0
    +    >>> n1 = 10
    +    >>> r2 = 4.0
    +    >>> n2 = 40
    +    >>> n = n1 + n2
    +    >>> points = genCircle(r1, n1) + genCircle(r2, n2)
    +    >>> similarities = []
    +    >>> for i in range (1, n):
    +    ...    neighbor = []
    +    ...    weight = []
    +    ...    for j in range (i):
    +    ...        neighbor.append((long)(j))
    +    ...        weight.append(sim(points[i], points[j]))
    +    ...    similarities.append([(long)(i), neighbor, weight])
    +    >>> rdd = sc.parallelize(similarities, 2)
    +    >>> schema = StructType([StructField("id", LongType(), False), \
    +             StructField("neighbors", ArrayType(LongType(), False), True), \
    +             StructField("similarities", ArrayType(DoubleType(), False), True)])
    +    >>> df = spark.createDataFrame(rdd, schema)
    +    >>> pic = PowerIterationClustering()
    +    >>> result = pic.setK(2).setMaxIter(40).transform(df)
    +    >>> predictions = sorted(set([(i[0], i[1]) for i in result.select(result.id, result.prediction)
    +    ...     .collect()]), key=lambda x: x[0])
    +    >>> predictions[0]
    +    (1, 1)
    +    >>> predictions[8]
    +    (9, 1)
    +    >>> predictions[9]
    +    (10, 0)
    +    >>> predictions[20]
    +    (21, 0)
    +    >>> predictions[48]
    +    (49, 0)
    +    >>> pic_path = temp_path + "/pic"
    +    >>> pic.save(pic_path)
    +    >>> pic2 = PowerIterationClustering.load(pic_path)
    +    >>> pic2.getK()
    +    2
    +    >>> pic2.getMaxIter()
    +    40
    +    >>> pic3 = PowerIterationClustering(k=4, initMode="degree")
    +    >>> pic3.getIdCol()
    +    'id'
    +    >>> pic3.getK()
    +    4
    +    >>> pic3.getMaxIter()
    +    20
    +    >>> pic3.getInitMode()
    +    'degree'
    +
    +    .. versionadded:: 2.4.0
    +    """
    +    @keyword_only
    +    def __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        super(PowerIterationClustering, self).__init__()
    +        self._java_obj = self._new_java_obj(
    +            "org.apache.spark.ml.clustering.PowerIterationClustering", self.uid)
    +        self._setDefault(k=2, maxIter=20, initMode="random", idCol="id", neighborsCol="neighbors",
    +                         similaritiesCol="similarities")
    +        kwargs = self._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    @keyword_only
    +    @since("2.4.0")
    +    def setParams(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                  idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        setParams(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                  idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    --- End diff --
    
    remove `:` at the end.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184874390
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,205 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +@inherit_doc
    +class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,
    +                               JavaMLReadable, JavaMLWritable):
    +    """
    +    .. note:: Experimental
    +
    +    Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
    +    <a href=http://www.icml2010.org/papers/387.pdf>Lin and Cohen</a>. From the abstract:
    +    PIC finds a very low-dimensional embedding of a dataset using truncated power
    +    iteration on a normalized pair-wise similarity matrix of the data.
    +
    +    PIC takes an affinity matrix between items (or vertices) as input.  An affinity matrix
    +    is a symmetric matrix whose entries are non-negative similarities between items.
    +    PIC takes this matrix (or graph) as an adjacency matrix.  Specifically, each input row
    +    includes:
    +
    +     - :py:attr:`idCol`: vertex ID
    +     - :py:attr:`neighborsCol`: neighbors of vertex in :py:attr:`idCol`
    +     - :py:attr:`similaritiesCol`: non-negative weights (similarities) of edges between the
    +        vertex in :py:attr:`idCol` and each neighbor in :py:attr:`neighborsCol`
    +
    +    PIC returns a cluster assignment for each input vertex.  It appends a new column
    +    :py:attr:`predictionCol` containing the cluster assignment in :py:attr:`[0,k)` for
    +    each row (vertex).
    +
    +    .. note::
    +
    +     - [[PowerIterationClustering]] is a transformer with an expensive [[transform]] operation.
    +        Transform runs the iterative PIC algorithm to cluster the whole input dataset.
    +     - Input validation: This validates that similarities are non-negative but does NOT validate
    +        that the input matrix is symmetric.
    +
    +    .. seealso:: <a href=http://en.wikipedia.org/wiki/Spectral_clustering>
    +    Spectral clustering (Wikipedia)</a>
    --- End diff --
    
    You can check other places using `seealso`:
    
    ```python
    .. seealso:: `Spectral clustering \
    <http://en.wikipedia.org/wiki/Spectral_clustering>`_
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89737 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89737/testReport)** for PR 21119 at commit [`387d6ff`](https://github.com/apache/spark/commit/387d6ffa8234f365e0de250082707187f67ecf61).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89970 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89970/testReport)** for PR 21119 at commit [`ae9f953`](https://github.com/apache/spark/commit/ae9f953d4a06228b6bf7b6867f031a1bfc84d1e2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184809072
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    +    """
    +    Params for :py:attr:`PowerIterationClustering`.
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of `idCol`
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def getNeighborsCol(self):
    +        """
    +        Gets the value of `neighborsCol`
    +        """
    +        return self.getOrDefault(self.neighborsCol)
    +
    +    @since("2.4.0")
    +    def getSimilaritiesCol(self):
    +        """
    +        Gets the value of `similaritiesCol`
    +        """
    +        return self.getOrDefault(self.binary)
    +
    +
    +@inherit_doc
    +class PowerIterationClustering(JavaTransformer, _PowerIterationClusteringParams, JavaMLReadable,
    +                               JavaMLWritable):
    +    """
    +    Model produced by [[PowerIterationClustering]].
    +    >>> from pyspark.sql.types import ArrayType, DoubleType, LongType, StructField, StructType
    +    >>> import math
    +    >>> def genCircle(r, n):
    +    ...     points = []
    +    ...     for i in range(0, n):
    +    ...         theta = 2.0 * math.pi * i / n
    +    ...         points.append((r * math.cos(theta), r * math.sin(theta)))
    +    ...     return points
    +    >>> def sim(x, y):
    +    ...     dist = (x[0] - y[0]) * (x[0] - y[0]) + (x[1] - y[1]) * (x[1] - y[1])
    +    ...     return math.exp(-dist / 2.0)
    +    >>> r1 = 1.0
    +    >>> n1 = 10
    +    >>> r2 = 4.0
    +    >>> n2 = 40
    +    >>> n = n1 + n2
    +    >>> points = genCircle(r1, n1) + genCircle(r2, n2)
    +    >>> similarities = []
    +    >>> for i in range (1, n):
    +    ...    neighbor = []
    +    ...    weight = []
    +    ...    for j in range (i):
    +    ...        neighbor.append((long)(j))
    +    ...        weight.append(sim(points[i], points[j]))
    +    ...    similarities.append([(long)(i), neighbor, weight])
    --- End diff --
    
    @WeichenXu123 I will move this to tests, and add a simple example in the doctest. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184839158
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,204 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +@inherit_doc
    +class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,
    +                               JavaMLReadable, JavaMLWritable):
    +    """
    +    .. note:: Experimental
    +    Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
    +    <a href=http://www.icml2010.org/papers/387.pdf>Lin and Cohen</a>. From the abstract:
    +    PIC finds a very low-dimensional embedding of a dataset using truncated power
    +    iteration on a normalized pair-wise similarity matrix of the data.
    +
    +    PIC takes an affinity matrix between items (or vertices) as input.  An affinity matrix
    +    is a symmetric matrix whose entries are non-negative similarities between items.
    +    PIC takes this matrix (or graph) as an adjacency matrix.  Specifically, each input row
    +    includes:
    +
    +     - :py:class:`idCol`: vertex ID
    +     - :py:class:`neighborsCol`: neighbors of vertex in :py:class:`idCol`
    +     - :py:class:`similaritiesCol`: non-negative weights (similarities) of edges between the
    +        vertex in :py:class:`idCol` and each neighbor in :py:class:`neighborsCol`
    +
    +    PIC returns a cluster assignment for each input vertex.  It appends a new column
    +    :py:class:`predictionCol` containing the cluster assignment in :py:class:`[0,k)` for
    +    each row (vertex).
    +
    +    Notes:
    --- End diff --
    
    Use `.. note::`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89970/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by WeichenXu123 <gi...@git.apache.org>.
Github user WeichenXu123 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184344901
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,201 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +class _PowerIterationClusteringParams(JavaParams, HasMaxIter, HasPredictionCol):
    +    """
    +    Params for :py:attr:`PowerIterationClustering`.
    +    .. versionadded:: 2.4.0
    +    """
    +
    +    k = Param(Params._dummy(), "k",
    +              "The number of clusters to create. Must be > 1.",
    +              typeConverter=TypeConverters.toInt)
    +    initMode = Param(Params._dummy(), "initMode",
    +                     "The initialization algorithm. This can be either " +
    +                     "'random' to use a random vector as vertex properties, or 'degree' to use " +
    +                     "a normalized sum of similarities with other vertices.  Supported options: " +
    +                     "'random' and 'degree'.",
    +                     typeConverter=TypeConverters.toString)
    +    idCol = Param(Params._dummy(), "idCol",
    +                  "Name of the input column for vertex IDs.",
    +                  typeConverter=TypeConverters.toString)
    +    neighborsCol = Param(Params._dummy(), "neighborsCol",
    +                         "Name of the input column for neighbors in the adjacency list " +
    +                         "representation.",
    +                         typeConverter=TypeConverters.toString)
    +    similaritiesCol = Param(Params._dummy(), "similaritiesCol",
    +                            "Name of the input column for non-negative weights (similarities) " +
    +                            "of edges between the vertex in `idCol` and each neighbor in " +
    +                            "`neighborsCol`",
    +                            typeConverter=TypeConverters.toString)
    +
    +    @since("2.4.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("2.4.0")
    +    def getInitMode(self):
    +        """
    +        Gets the value of `initMode`
    +        """
    +        return self.getOrDefault(self.initMode)
    +
    +    @since("2.4.0")
    +    def getIdCol(self):
    +        """
    +        Gets the value of `idCol`
    +        """
    +        return self.getOrDefault(self.idCol)
    +
    +    @since("2.4.0")
    +    def getNeighborsCol(self):
    +        """
    +        Gets the value of `neighborsCol`
    +        """
    +        return self.getOrDefault(self.neighborsCol)
    +
    +    @since("2.4.0")
    +    def getSimilaritiesCol(self):
    +        """
    +        Gets the value of `similaritiesCol`
    +        """
    +        return self.getOrDefault(self.binary)
    +
    +
    +@inherit_doc
    +class PowerIterationClustering(JavaTransformer, _PowerIterationClusteringParams, JavaMLReadable,
    +                               JavaMLWritable):
    +    """
    +    Model produced by [[PowerIterationClustering]].
    +    >>> from pyspark.sql.types import ArrayType, DoubleType, LongType, StructField, StructType
    +    >>> import math
    +    >>> def genCircle(r, n):
    +    ...     points = []
    +    ...     for i in range(0, n):
    +    ...         theta = 2.0 * math.pi * i / n
    +    ...         points.append((r * math.cos(theta), r * math.sin(theta)))
    +    ...     return points
    +    >>> def sim(x, y):
    +    ...     dist = (x[0] - y[0]) * (x[0] - y[0]) + (x[1] - y[1]) * (x[1] - y[1])
    +    ...     return math.exp(-dist / 2.0)
    +    >>> r1 = 1.0
    +    >>> n1 = 10
    +    >>> r2 = 4.0
    +    >>> n2 = 40
    +    >>> n = n1 + n2
    +    >>> points = genCircle(r1, n1) + genCircle(r2, n2)
    +    >>> similarities = []
    +    >>> for i in range (1, n):
    +    ...    neighbor = []
    +    ...    weight = []
    +    ...    for j in range (i):
    +    ...        neighbor.append((long)(j))
    +    ...        weight.append(sim(points[i], points[j]))
    +    ...    similarities.append([(long)(i), neighbor, weight])
    +    >>> rdd = sc.parallelize(similarities, 2)
    +    >>> schema = StructType([StructField("id", LongType(), False), \
    +             StructField("neighbors", ArrayType(LongType(), False), True), \
    +             StructField("similarities", ArrayType(DoubleType(), False), True)])
    +    >>> df = spark.createDataFrame(rdd, schema)
    +    >>> pic = PowerIterationClustering()
    +    >>> result = pic.setK(2).setMaxIter(40).transform(df)
    +    >>> predictions = sorted(set([(i[0], i[1]) for i in result.select(result.id, result.prediction)
    +    ...     .collect()]), key=lambda x: x[0])
    +    >>> predictions[0]
    +    (1, 1)
    +    >>> predictions[8]
    +    (9, 1)
    +    >>> predictions[9]
    +    (10, 0)
    +    >>> predictions[20]
    +    (21, 0)
    +    >>> predictions[48]
    +    (49, 0)
    +    >>> pic_path = temp_path + "/pic"
    +    >>> pic.save(pic_path)
    +    >>> pic2 = PowerIterationClustering.load(pic_path)
    +    >>> pic2.getK()
    +    2
    +    >>> pic2.getMaxIter()
    +    40
    +    >>> pic3 = PowerIterationClustering(k=4, initMode="degree")
    +    >>> pic3.getIdCol()
    +    'id'
    +    >>> pic3.getK()
    +    4
    +    >>> pic3.getMaxIter()
    +    20
    +    >>> pic3.getInitMode()
    +    'degree'
    +
    +    .. versionadded:: 2.4.0
    +    """
    +    @keyword_only
    +    def __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    +        """
    +        __init__(self, predictionCol="prediction", k=2, maxIter=20, initMode="random",\
    +                 idCol="id", neighborsCol="neighbors", similaritiesCol="similarities"):
    --- End diff --
    
    remove `:` at the end.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89735 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89735/testReport)** for PR 21119 at commit [`2d0e394`](https://github.com/apache/spark/commit/2d0e3943440718c7f603cdaba410ba35ab81279e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/89672/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89946 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89946/testReport)** for PR 21119 at commit [`a6b1822`](https://github.com/apache/spark/commit/a6b18222b65e878e22ddf8f2d340aa3127c99e0c).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao closed the pull request at:

    https://github.com/apache/spark/pull/21119


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API ...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21119#discussion_r184839128
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -1156,6 +1156,204 @@ def getKeepLastCheckpoint(self):
             return self.getOrDefault(self.keepLastCheckpoint)
     
     
    +@inherit_doc
    +class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,
    +                               JavaMLReadable, JavaMLWritable):
    +    """
    +    .. note:: Experimental
    +    Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
    +    <a href=http://www.icml2010.org/papers/387.pdf>Lin and Cohen</a>. From the abstract:
    +    PIC finds a very low-dimensional embedding of a dataset using truncated power
    +    iteration on a normalized pair-wise similarity matrix of the data.
    +
    +    PIC takes an affinity matrix between items (or vertices) as input.  An affinity matrix
    +    is a symmetric matrix whose entries are non-negative similarities between items.
    +    PIC takes this matrix (or graph) as an adjacency matrix.  Specifically, each input row
    +    includes:
    +
    +     - :py:class:`idCol`: vertex ID
    --- End diff --
    
    ```:py:attr:`idCol` ```? And also the below ```:py:class:`neighborsCol` ```, etc...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by huaxingao <gi...@git.apache.org>.
Github user huaxingao commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    @mengxr @WeichenXu123 I will update this. Thanks. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/2597/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89943 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89943/testReport)** for PR 21119 at commit [`6d00f34`](https://github.com/apache/spark/commit/6d00f343f5c78fbe290793fe85cbc3deed53cf3e).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class PowerIterationClustering(HasMaxIter, HasPredictionCol, JavaTransformer, JavaParams,`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21119: [SPARK-19826][ML][PYTHON]add spark.ml Python API for PIC

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21119
  
    **[Test build #89735 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/89735/testReport)** for PR 21119 at commit [`2d0e394`](https://github.com/apache/spark/commit/2d0e3943440718c7f603cdaba410ba35ab81279e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org