You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by zjffdu <gi...@git.apache.org> on 2015/12/10 04:30:11 UTC

[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

GitHub user zjffdu opened a pull request:

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

    [SPARK-11940][PYSPARK] Python API for ml.clustering.LDA

    Besides this issue, also fix another in issue in python/pyspark/__init__.py (should provide more informative message when no doc is defined but since annotation is added. 
    
      


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

    $ git pull https://github.com/zjffdu/spark SPARK-11940

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

    https://github.com/apache/spark/pull/10242.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 #10242
    
----
commit 792b883c94e2a693ba7bb25a0bfd56b7bc1c61fb
Author: Jeff Zhang <zj...@apache.org>
Date:   2015-12-04T14:50:08Z

    [SPARK-11940][PYSPARK] Python API for ml.clustering.LDA

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57895188
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    +                              "Output column with estimates of the topic mixture distribution for "
    +                              "each document (often called \"theta\" in the literature). Returns "
    +                              "a vector of zeros for an empty document.")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    --- End diff --
    
    Actually we set default value for ```topicDistributionCol``` at [```setDefault(topicDistributionCol -> "topicDistribution")```](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala#L183)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by zjffdu <gi...@git.apache.org>.
Github user zjffdu commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-188539283
  
    Sorry for late response, I will update this PR in the next following days. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470033
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    --- End diff --
    
    Could you please add the ```trainingLogLikelihood, logPrior, getCheckpointFiles``` methods to this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-190107969
  
    **[Test build #52180 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52180/consoleFull)** for PR 10242 at commit [`b27c275`](https://github.com/apache/spark/commit/b27c2752e25099914fe5536ce2ff250e2f9037e6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class LDAModel(JavaModel):`
      * `class DistributedLDAModel(LDAModel):`
      * `class LocalLDAModel(LDAModel):`
      * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57901007
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    --- End diff --
    
    Hmm... I think this is a bug at Scala side, so I send #12065 to fix it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60680711
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -59,6 +59,8 @@ def since(version):
         indent_p = re.compile(r'\n( +)')
     
         def deco(f):
    +        if not f.__doc__:
    --- End diff --
    
    create SPARK-14834 for this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-200169823
  
    **[Test build #53888 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53888/consoleFull)** for PR 10242 at commit [`d6c9078`](https://github.com/apache/spark/commit/d6c90781153ed72802f722abbbc6dd4cb2ff1cd8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47619640
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=2,
    +                 optimizer="online", learningOffset=5, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", predictionCol="prediction", k=2, \
    +                 initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None)
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer", "LDA optimizer")
    +        self._setDefault(k=2, optimizer="online", maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("1.7.0")
    +    def setParams(self, featuresCol="features", k=2,
    +                  optimizer="online", learningOffset=5, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        ssetParams(self, featuresCol="features", k=2,
    --- End diff --
    
    
    NAVER - http://www.naver.com/
    --------------------------------------------
    
    3ourroom@naver.com 님께 보내신 메일 <Re: [spark] [SPARK-11940][PYSPARK] Python API for ml.clustering.LDA (#10242)> 이 다음과 같은 이유로 전송 실패했습니다.
    
    --------------------------------------------
    
    받는 사람이 회원님의 메일을 수신차단 하였습니다. 
    
    
    --------------------------------------------



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165000395
  
    **[Test build #47790 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47790/consoleFull)** for PR 10242 at commit [`ebf5e35`](https://github.com/apache/spark/commit/ebf5e35d3eda97c279c2a57791e6878cc38b054f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47621148
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=2,
    +                 optimizer="online", learningOffset=5, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", predictionCol="prediction", k=2, \
    +                 initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None)
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer", "LDA optimizer")
    --- End diff --
    
    You should define ```learningOffset```, ```learningDecay```, ```subsamplingRate```, ```optimizeDocConcentration``` and their default values, then use can set these parameters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-196663574
  
    Great, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54394752
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    --- End diff --
    
    Remove this line because it's not only placeholder after #10216.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-200170035
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#issuecomment-214918422
  
    Build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212274330
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r56450720
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -21,8 +21,8 @@
     from pyspark.ml.param.shared import *
     from pyspark.mllib.common import inherit_doc
     
    -__all__ = ['BisectingKMeans', 'BisectingKMeansModel',
    -           'KMeans', 'KMeansModel']
    +__all__ = ['KMeans', 'KMeansModel', 'BisectingKMeans', 'BisectingKMeansModel',
    +           'LDA', 'LDAModel', 'LocalLDAModel', 'DistributedLDAModel']
    --- End diff --
    
    Put one alg per line, see #10927


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165024285
  
    **[Test build #47801 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47801/consoleFull)** for PR 10242 at commit [`592f50b`](https://github.com/apache/spark/commit/592f50bf087205022d1cb4e389d44484b8c91124).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by yanboliang <gi...@git.apache.org>.
Github user yanboliang commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-212448497
  
    @zjffdu Please update this PR according the change at #11663 for type conversion and #11939 for param setters using ```_set``` method. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54395410
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffSet = Param(Params._dummy(), "learningOffSet",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        :param featureCol:          feature column name in the dataframe
    +        :param k:                   Number of clusters you want
    +        :param optimizer:           LDAOptimizer used to perform the actual calculation.
    +            Currently "em", "online" are supported. Default to "em".
    +
    +        :param learningOffset:      A (positive) learning parameter that downweights early
    +            iterations. Larger values make early iterations count less. This is called "tau0"
    +            in the Online LDA paper (Hoffman et al., 2010) Default: 1024.0, following Hoffman et al.
    +        :param learningDecay:       Learning rate, set as an exponential decay rate. This should
    +            be between (0.5, 1.0] to guarantee asymptotic convergence. This is called "kappa" in
    +            the Online LDA paper (Hoffman et al., 2010). Default: 0.51, based on Hoffman et al.
    +        :param subsamplingRate:     Fraction of the corpus to be sampled and used in each iteration
    +            of mini-batch gradient descent, in range (0, 1]. Note that this should be adjusted in
    +            synch with [[LDA.maxIter]] so the entire corpus is used.  Specifically, set both so that
    +            maxIterations * miniBatchFraction >= 1. Default:0.05
    +        :param optimizeDocConcentration:       Indicates whether the docConcentration (Dirichlet
    +            parameter for document-topic distribution) will be optimized during training. Setting
    +            this to true will make the model more expressive and fit the training data better.
    +            Default: false
    +        :param checkpointInterval:  Param for set checkpoint interval (>= 1) or disable
    +            checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
    +        :param maxIter:             Number of iterations. Default to 20
    +        :param seed:                Param for random seed
    --- End diff --
    
    We don't need to doc param here, L407 - L417 can help these params appear in the generated doc already.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165014295
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54394942
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffSet = Param(Params._dummy(), "learningOffSet",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        :param featureCol:          feature column name in the dataframe
    +        :param k:                   Number of clusters you want
    +        :param optimizer:           LDAOptimizer used to perform the actual calculation.
    +            Currently "em", "online" are supported. Default to "em".
    +
    +        :param learningOffset:      A (positive) learning parameter that downweights early
    +            iterations. Larger values make early iterations count less. This is called "tau0"
    +            in the Online LDA paper (Hoffman et al., 2010) Default: 1024.0, following Hoffman et al.
    +        :param learningDecay:       Learning rate, set as an exponential decay rate. This should
    +            be between (0.5, 1.0] to guarantee asymptotic convergence. This is called "kappa" in
    +            the Online LDA paper (Hoffman et al., 2010). Default: 0.51, based on Hoffman et al.
    +        :param subsamplingRate:     Fraction of the corpus to be sampled and used in each iteration
    +            of mini-batch gradient descent, in range (0, 1]. Note that this should be adjusted in
    +            synch with [[LDA.maxIter]] so the entire corpus is used.  Specifically, set both so that
    +            maxIterations * miniBatchFraction >= 1. Default:0.05
    +        :param optimizeDocConcentration:       Indicates whether the docConcentration (Dirichlet
    +            parameter for document-topic distribution) will be optimized during training. Setting
    +            this to true will make the model more expressive and fit the training data better.
    +            Default: false
    +        :param checkpointInterval:  Param for set checkpoint interval (>= 1) or disable
    +            checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
    +        :param maxIter:             Number of iterations. Default to 20
    +        :param seed:                Param for random seed
    +        __init__(self, featuresCol="features", k=10, \
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                 subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer",
    +                               "LDA optimizer, Currently 'em', 'online' are supported.")
    +        self.learningOffset = Param(self, "learningOffset",
    +                                    "learning parameter that downweights early iterations")
    +        self.learningDecay = Param(self, "learningDecay", "Learning rate")
    +        self.subsamplingRate = Param(self, "subsamplingRate",
    +                                     "Fraction of the corpus to be sampled")
    +        self.optimizeDocConcentration = Param(self, "optimizeDocConcentration",
    --- End diff --
    
    It's not necessary to generate Params after #10216. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54695344
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    --- End diff --
    
    We should keep Python param doc consistent with Scala one, so here should be ```number of topics (clusters) to infer```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54694890
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    --- End diff --
    
    L370-379 is unnecessary, is mainly used to describe ```LDAModel``` and we have these documents at L296-305.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by zjffdu <gi...@git.apache.org>.
Github user zjffdu commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-212171944
  
    Sorry for late response, will do it today. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-196761978
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47621769
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    --- End diff --
    
    ```\``` is unnecessary


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54395971
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    --- End diff --
    
    ```.. versionadded:: 2.0.0``` for ```LDAModel```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57894849
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    +                              "Output column with estimates of the topic mixture distribution for "
    +                              "each document (often called \"theta\" in the literature). Returns "
    +                              "a vector of zeros for an empty document.")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    --- End diff --
    
    The arguments of ```__init__``` are not params which must have default values, it should include all params of the Estimator/Model. Take the ```LogisticRegression``` as example, ```thresholds``` has no default value, but it also in the arguments of ```__init__```. 
    For default values, we use ```self._setDefault()``` to set default value for params and the default values were stored in a separate map named ```_defaultParamMap``` which is different from the params map. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47621638
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=2,
    +                 optimizer="online", learningOffset=5, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", predictionCol="prediction", k=2, \
    +                 initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None)
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer", "LDA optimizer")
    +        self._setDefault(k=2, optimizer="online", maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("1.7.0")
    +    def setParams(self, featuresCol="features", k=2,
    +                  optimizer="online", learningOffset=5, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        ssetParams(self, featuresCol="features", k=2,
    +                  optimizer="online", learningOffset=5, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +
    +        Sets params for LDA.
    +        """
    +        kwargs = self.setParams._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("1.7.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +
    +        >>> algo = LDA().setK(10)
    +        >>> algo.getK()
    +        10
    +        """
    +        self._paramMap[self.k] = value
    +        return self
    +
    +    @since("1.7.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    --- End diff --
    
    nit: ```Gets the value of k or its default value.```
    I guess you refer python/pyspark/ml/clustering.py, maybe we should also update that class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54665060
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffSet = Param(Params._dummy(), "learningOffSet",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        :param featureCol:          feature column name in the dataframe
    +        :param k:                   Number of clusters you want
    +        :param optimizer:           LDAOptimizer used to perform the actual calculation.
    +            Currently "em", "online" are supported. Default to "em".
    +
    +        :param learningOffset:      A (positive) learning parameter that downweights early
    +            iterations. Larger values make early iterations count less. This is called "tau0"
    +            in the Online LDA paper (Hoffman et al., 2010) Default: 1024.0, following Hoffman et al.
    +        :param learningDecay:       Learning rate, set as an exponential decay rate. This should
    +            be between (0.5, 1.0] to guarantee asymptotic convergence. This is called "kappa" in
    +            the Online LDA paper (Hoffman et al., 2010). Default: 0.51, based on Hoffman et al.
    +        :param subsamplingRate:     Fraction of the corpus to be sampled and used in each iteration
    +            of mini-batch gradient descent, in range (0, 1]. Note that this should be adjusted in
    +            synch with [[LDA.maxIter]] so the entire corpus is used.  Specifically, set both so that
    +            maxIterations * miniBatchFraction >= 1. Default:0.05
    +        :param optimizeDocConcentration:       Indicates whether the docConcentration (Dirichlet
    +            parameter for document-topic distribution) will be optimized during training. Setting
    +            this to true will make the model more expressive and fit the training data better.
    +            Default: false
    +        :param checkpointInterval:  Param for set checkpoint interval (>= 1) or disable
    +            checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
    +        :param maxIter:             Number of iterations. Default to 20
    +        :param seed:                Param for random seed
    +        __init__(self, featuresCol="features", k=10, \
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                 subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer",
    +                               "LDA optimizer, Currently 'em', 'online' are supported.")
    +        self.learningOffset = Param(self, "learningOffset",
    +                                    "learning parameter that downweights early iterations")
    +        self.learningDecay = Param(self, "learningDecay", "Learning rate")
    +        self.subsamplingRate = Param(self, "subsamplingRate",
    +                                     "Fraction of the corpus to be sampled")
    +        self.optimizeDocConcentration = Param(self, "optimizeDocConcentration",
    +                                              "Indicates whether the docConcentration \
    +                                              will be optimized during training")
    +        self._setDefault(k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                         subsamplingRate=0.05, optimizeDocConcentration=True,
    +                         checkpointInterval=10, maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("2.0.0")
    +    def setParams(self, featuresCol="features", k=10,
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        setParams(self, featuresCol="features", k=10, \
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                  subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +
    +        Sets params for LDA.
    +        """
    +        kwargs = self.setParams._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("2.0.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +
    +        >>> algo = LDA().setK(10)
    +        >>> algo.getK()
    +        10
    --- End diff --
    
    I take a look at #10975, but think these code are not only test code, but can also be treated as sample code for user to refer. Sometimes code is more intuitive than comment. What do you think ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163858547
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54394622
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    --- End diff --
    
    Here we usually make the next line start with ```...```, you can refer [here](https://github.com/apache/spark/blob/master/python/pyspark/ml/clustering.py#L58).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by zjffdu <gi...@git.apache.org>.
Github user zjffdu commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-164351234
  
    @yanboliang Could you help review it ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by yanboliang <gi...@git.apache.org>.
Github user yanboliang commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-197754704
  
    @zjffdu The PR looks good overall, please add the three parameters and corresponding set/get method. Thanks! 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60471281
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    --- End diff --
    
    This is not applicable to EM.  You can just remove these 2 lines.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212558479
  
    @zjffdu thanks for the updates!  Btw, can you please add the "[ML]" tag to the PR title?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470025
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    --- End diff --
    
    "terms or terms" must be a mistake from a search-and-replace.  I bet it's supposed to be "terms or words"
    Could you fix that here and in the Scala doc too please?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-197199809
  
    @yanboliang Is this PR ready for merge?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-196620183
  
    Ping--any updates?  It will be great to get this merged for 2.0 if possible


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163478325
  
    **[Test build #47480 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47480/consoleFull)** for PR 10242 at commit [`792b883`](https://github.com/apache/spark/commit/792b883c94e2a693ba7bb25a0bfd56b7bc1c61fb).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470044
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    --- End diff --
    
    Does this work?  You need to create a Python LocalLDAModel which wraps the Java object


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-196753549
  
    **[Test build #53188 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53188/consoleFull)** for PR 10242 at commit [`4ab456c`](https://github.com/apache/spark/commit/4ab456c3af9985afdd1a9d218a40b3fb61b476f1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47620659
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    --- End diff --
    
    More detail documents(such as currently supported optimizer) needed, it will appeared in the Python API doc. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-200164987
  
    **[Test build #53888 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53888/consoleFull)** for PR 10242 at commit [`d6c9078`](https://github.com/apache/spark/commit/d6c90781153ed72802f722abbbc6dd4cb2ff1cd8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163485515
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57837880
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    --- End diff --
    
    Params require the declared field name same as the value of its field name, otherwise we will get error. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54696699
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffSet = Param(Params._dummy(), "learningOffSet",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        :param featureCol:          feature column name in the dataframe
    +        :param k:                   Number of clusters you want
    +        :param optimizer:           LDAOptimizer used to perform the actual calculation.
    +            Currently "em", "online" are supported. Default to "em".
    +
    +        :param learningOffset:      A (positive) learning parameter that downweights early
    +            iterations. Larger values make early iterations count less. This is called "tau0"
    +            in the Online LDA paper (Hoffman et al., 2010) Default: 1024.0, following Hoffman et al.
    +        :param learningDecay:       Learning rate, set as an exponential decay rate. This should
    +            be between (0.5, 1.0] to guarantee asymptotic convergence. This is called "kappa" in
    +            the Online LDA paper (Hoffman et al., 2010). Default: 0.51, based on Hoffman et al.
    +        :param subsamplingRate:     Fraction of the corpus to be sampled and used in each iteration
    +            of mini-batch gradient descent, in range (0, 1]. Note that this should be adjusted in
    +            synch with [[LDA.maxIter]] so the entire corpus is used.  Specifically, set both so that
    +            maxIterations * miniBatchFraction >= 1. Default:0.05
    +        :param optimizeDocConcentration:       Indicates whether the docConcentration (Dirichlet
    +            parameter for document-topic distribution) will be optimized during training. Setting
    +            this to true will make the model more expressive and fit the training data better.
    +            Default: false
    +        :param checkpointInterval:  Param for set checkpoint interval (>= 1) or disable
    +            checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
    +        :param maxIter:             Number of iterations. Default to 20
    +        :param seed:                Param for random seed
    +        __init__(self, featuresCol="features", k=10, \
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                 subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer",
    +                               "LDA optimizer, Currently 'em', 'online' are supported.")
    +        self.learningOffset = Param(self, "learningOffset",
    +                                    "learning parameter that downweights early iterations")
    +        self.learningDecay = Param(self, "learningDecay", "Learning rate")
    +        self.subsamplingRate = Param(self, "subsamplingRate",
    +                                     "Fraction of the corpus to be sampled")
    +        self.optimizeDocConcentration = Param(self, "optimizeDocConcentration",
    +                                              "Indicates whether the docConcentration \
    +                                              will be optimized during training")
    +        self._setDefault(k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                         subsamplingRate=0.05, optimizeDocConcentration=True,
    +                         checkpointInterval=10, maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("2.0.0")
    +    def setParams(self, featuresCol="features", k=10,
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        setParams(self, featuresCol="features", k=10, \
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                  subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +
    +        Sets params for LDA.
    +        """
    +        kwargs = self.setParams._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("2.0.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +
    +        >>> algo = LDA().setK(10)
    +        >>> algo.getK()
    +        10
    --- End diff --
    
    I think the ```set/get``` method is easy to understand for users. I vote to move these codes to tests is to keep consistent code style with other classes in ML. Anyway, this is a minor issue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-200143226
  
    **[Test build #53872 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53872/consoleFull)** for PR 10242 at commit [`b9b08af`](https://github.com/apache/spark/commit/b9b08af5ee66a55ea1fa7cddc36d690e60ab2fc3).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57450611
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    --- End diff --
    
    This should be ```topicDistributionCol```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by zjffdu <gi...@git.apache.org>.
Github user zjffdu commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-196621006
  
    Sorry for late response, will update it today.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212244489
  
    **[Test build #56318 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56318/consoleFull)** for PR 10242 at commit [`16ea17d`](https://github.com/apache/spark/commit/16ea17dc0c9f8485f8856fc722f8ab4d91e63da0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by yanboliang <gi...@git.apache.org>.
Github user yanboliang commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-201319739
  
    Looks good for me. cc @mengxr @jkbradley 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#issuecomment-214918410
  
    **[Test build #57050 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57050/consoleFull)** for PR 10242 at commit [`2b2bafe`](https://github.com/apache/spark/commit/2b2bafe28e9da16b722f1bc7161620e20fdce1cf).
     * This patch **fails Python style tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163478251
  
    **[Test build #47480 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47480/consoleFull)** for PR 10242 at commit [`792b883`](https://github.com/apache/spark/commit/792b883c94e2a693ba7bb25a0bfd56b7bc1c61fb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57451032
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    +                              "Output column with estimates of the topic mixture distribution for "
    +                              "each document (often called \"theta\" in the literature). Returns "
    +                              "a vector of zeros for an empty document.")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    --- End diff --
    
    ```docConcentration, topicConcentration, topicDistributionCol``` should be params of ```__init__```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54695541
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    --- End diff --
    
    ```docConcentration``` should also a param.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54395661
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffSet = Param(Params._dummy(), "learningOffSet",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        :param featureCol:          feature column name in the dataframe
    +        :param k:                   Number of clusters you want
    +        :param optimizer:           LDAOptimizer used to perform the actual calculation.
    +            Currently "em", "online" are supported. Default to "em".
    +
    +        :param learningOffset:      A (positive) learning parameter that downweights early
    +            iterations. Larger values make early iterations count less. This is called "tau0"
    +            in the Online LDA paper (Hoffman et al., 2010) Default: 1024.0, following Hoffman et al.
    +        :param learningDecay:       Learning rate, set as an exponential decay rate. This should
    +            be between (0.5, 1.0] to guarantee asymptotic convergence. This is called "kappa" in
    +            the Online LDA paper (Hoffman et al., 2010). Default: 0.51, based on Hoffman et al.
    +        :param subsamplingRate:     Fraction of the corpus to be sampled and used in each iteration
    +            of mini-batch gradient descent, in range (0, 1]. Note that this should be adjusted in
    +            synch with [[LDA.maxIter]] so the entire corpus is used.  Specifically, set both so that
    +            maxIterations * miniBatchFraction >= 1. Default:0.05
    +        :param optimizeDocConcentration:       Indicates whether the docConcentration (Dirichlet
    +            parameter for document-topic distribution) will be optimized during training. Setting
    +            this to true will make the model more expressive and fit the training data better.
    +            Default: false
    +        :param checkpointInterval:  Param for set checkpoint interval (>= 1) or disable
    +            checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
    +        :param maxIter:             Number of iterations. Default to 20
    +        :param seed:                Param for random seed
    +        __init__(self, featuresCol="features", k=10, \
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                 subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer",
    +                               "LDA optimizer, Currently 'em', 'online' are supported.")
    +        self.learningOffset = Param(self, "learningOffset",
    +                                    "learning parameter that downweights early iterations")
    +        self.learningDecay = Param(self, "learningDecay", "Learning rate")
    +        self.subsamplingRate = Param(self, "subsamplingRate",
    +                                     "Fraction of the corpus to be sampled")
    +        self.optimizeDocConcentration = Param(self, "optimizeDocConcentration",
    +                                              "Indicates whether the docConcentration \
    +                                              will be optimized during training")
    +        self._setDefault(k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                         subsamplingRate=0.05, optimizeDocConcentration=True,
    +                         checkpointInterval=10, maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("2.0.0")
    +    def setParams(self, featuresCol="features", k=10,
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        setParams(self, featuresCol="features", k=10, \
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                  subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +
    +        Sets params for LDA.
    +        """
    +        kwargs = self.setParams._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("2.0.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +
    +        >>> algo = LDA().setK(10)
    +        >>> algo.getK()
    +        10
    --- End diff --
    
    Minor: It's better to add these tests to unit test files, you can refer #10975.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47621675
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=2,
    +                 optimizer="online", learningOffset=5, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", predictionCol="prediction", k=2, \
    +                 initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None)
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer", "LDA optimizer")
    +        self._setDefault(k=2, optimizer="online", maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("1.7.0")
    +    def setParams(self, featuresCol="features", k=2,
    +                  optimizer="online", learningOffset=5, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        ssetParams(self, featuresCol="features", k=2,
    +                  optimizer="online", learningOffset=5, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +
    +        Sets params for LDA.
    +        """
    +        kwargs = self.setParams._input_kwargs
    +        return self._set(**kwargs)
    +
    +    @since("1.7.0")
    +    def setK(self, value):
    +        """
    +        Sets the value of :py:attr:`k`.
    +
    +        >>> algo = LDA().setK(10)
    +        >>> algo.getK()
    +        10
    +        """
    +        self._paramMap[self.k] = value
    +        return self
    +
    +    @since("1.7.0")
    +    def getK(self):
    +        """
    +        Gets the value of `k`
    +        """
    +        return self.getOrDefault(self.k)
    +
    +    @since("1.7.0")
    +    def setOptimizer(self, value):
    +        """
    +        Sets the value of :py:attr:`optimizer`.
    +
    +        >>> algo = LDA().setOptimizer("em")
    +        >>> algo.getOptimizer()
    +        'em'
    +        """
    +        self._paramMap[self.optimizer] = value
    +        return self
    +
    +    @since("1.7.0")
    +    def getOptimizer(self):
    +        """
    +        Gets the value of `optimizer`
    --- End diff --
    
    ditto


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r56468043
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,312 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    --- End diff --
    
    ```docConcentration, topicConcentration, topicDistributionCol``` should also as parameters and can be ```set/get```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#issuecomment-214863595
  
    @zjffdu Do you mind if I take over this PR?  I'd really like to get this API in for 2.0.  You'll still be the primary author on the commit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212157155
  
    @zjffdu Will you have time to continue working on this?  If not, others can help get it into 2.0.  Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470027
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    --- End diff --
    
    No need for this warning anymore since maxTermsPerTopic is small now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212244679
  
    **[Test build #56318 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56318/consoleFull)** for PR 10242 at commit [`16ea17d`](https://github.com/apache/spark/commit/16ea17dc0c9f8485f8856fc722f8ab4d91e63da0).
     * This patch **fails Python style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-191008294
  
    **[Test build #52277 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52277/consoleFull)** for PR 10242 at commit [`e8723db`](https://github.com/apache/spark/commit/e8723db61badc72ae16d533bf7154c917bfd61de).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212267099
  
    **[Test build #56323 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56323/consoleFull)** for PR 10242 at commit [`372d5a5`](https://github.com/apache/spark/commit/372d5a5ff5effd63bb103a784f3a1872b49ff6e5).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470039
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    --- End diff --
    
    Copy doc from Scala for this class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

Posted by zjffdu <gi...@git.apache.org>.
Github user zjffdu commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-214918065
  
    @jkbradley I made some update based your comments before, but don't have time to implement the model persistence feature. Please take over this PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165020518
  
    **[Test build #47797 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47797/consoleFull)** for PR 10242 at commit [`bef9c91`](https://github.com/apache/spark/commit/bef9c9148ba3befa9b716e48fb2af916e8d15e20).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163478327
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-197708413
  
    **[Test build #53395 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53395/consoleFull)** for PR 10242 at commit [`5d75376`](https://github.com/apache/spark/commit/5d75376b0c65069fe26fc172ece32b01f6357d2b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165028026
  
    **[Test build #47801 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47801/consoleFull)** for PR 10242 at commit [`592f50b`](https://github.com/apache/spark/commit/592f50bf087205022d1cb4e389d44484b8c91124).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-183143332
  
    @zjffdu Sorry for slow response! @zjffdu Could you update since versions in this PR and address @yanboliang 's comment? Next version will be `2.0.0` instead of `1.7.0`.
    
    @yanboliang Could you make another pass after the update? Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470048
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    --- End diff --
    
    Copy doc from Scala for this class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-191016818
  
    **[Test build #52277 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52277/consoleFull)** for PR 10242 at commit [`e8723db`](https://github.com/apache/spark/commit/e8723db61badc72ae16d533bf7154c917bfd61de).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54695408
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    --- End diff --
    
    Ditto


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212559101
  
    One more high-level request: Could you please add persistence to this?  I'd like to start adding persistence to Python wrappers immediately since we now have full Python coverage.  You should be able to extend MLReadable, MLWritable and add a simple test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47619302
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=2,
    +                 optimizer="online", learningOffset=5, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", predictionCol="prediction", k=2, \
    +                 initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None)
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self.k = Param(self, "k", "number of clusters to create")
    +        self.optimizer = Param(self, "optimizer", "LDA optimizer")
    +        self._setDefault(k=2, optimizer="online", maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("1.7.0")
    +    def setParams(self, featuresCol="features", k=2,
    +                  optimizer="online", learningOffset=5, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        ssetParams(self, featuresCol="features", k=2,
    --- End diff --
    
    add ```\``` at the end of each line. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212244685
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165028129
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-190108128
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54700006
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    --- End diff --
    
    L296-305 is unnecessary, we have these documents at L370-379.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-200143242
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#discussion_r60683456
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    --- End diff --
    
    Also, you will need to modify spacing to make the bullets show up properly.  newline before bulleted list + indent bullets by a space.  E.g.:
    ```
        Terminology
    
         - "word" = "term": an element of the vocabulary
         - "token": instance of a term appearing in a document
         - "topic": multinomial distribution over words representing some concept
    ```
    
    You can test it in spark/python/docs by running "make html"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57971368
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    --- End diff --
    
    I just merged [https://github.com/apache/spark/pull/12065]


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60471284
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistributionCol = Param(Params._dummy(), "topicDistributionCol",
    +                                 "Output column with estimates of the topic mixture distribution "
    +                                 "for each document (often called \"theta\" in the literature). "
    +                                 "Returns a vector of zeros for an empty document.")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, docConcentration=None,
    +                 topicConcentration=None, topicDistributionCol="topicDistribution", seed=None):
    +        """
    +        __init__(self, featuresCol="features", k=10, \
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                 subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                 checkpointInterval=10, maxIter=20, docConcentration=None, \
    +                 topicConcentration=None, topicDistributionCol="topicDistribution", seed=None):
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self._setDefault(k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                         subsamplingRate=0.05, optimizeDocConcentration=True,
    +                         checkpointInterval=10, maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("2.0.0")
    +    def setParams(self, featuresCol="features", k=10,
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, docConcentration=None,
    +                  topicConcentration=None,
    +                  topicDistributionCol="topicDistribution", seed=None):
    +        """
    +        setParams(self, featuresCol="features", k=10, \
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                  subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                  checkpointInterval=10, maxIter=20, docConcentration=None,
    --- End diff --
    
    need backslash on each line of doc here for continuations to show up properly ```\```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-191017019
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-164988137
  
    **[Test build #47788 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47788/consoleFull)** for PR 10242 at commit [`d189853`](https://github.com/apache/spark/commit/d189853601cde828c307e1d8c2f22cf44126c1e6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-197702857
  
    **[Test build #53395 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53395/consoleFull)** for PR 10242 at commit [`5d75376`](https://github.com/apache/spark/commit/5d75376b0c65069fe26fc172ece32b01f6357d2b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-197708603
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163485489
  
    **[Test build #47483 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47483/consoleFull)** for PR 10242 at commit [`5b5f091`](https://github.com/apache/spark/commit/5b5f091c4f0a0f0ead854d46970dcec7a0350bc6).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163483623
  
    **[Test build #47483 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47483/consoleFull)** for PR 10242 at commit [`5b5f091`](https://github.com/apache/spark/commit/5b5f091c4f0a0f0ead854d46970dcec7a0350bc6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54694352
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    --- End diff --
    
    Should we also expose ```estimatedDocConcentration``` for ```LDAModel```?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-164991821
  
    **[Test build #47788 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47788/consoleFull)** for PR 10242 at commit [`d189853`](https://github.com/apache/spark/commit/d189853601cde828c307e1d8c2f22cf44126c1e6).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60467216
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -59,6 +59,8 @@ def since(version):
         indent_p = re.compile(r'\n( +)')
     
         def deco(f):
    +        if not f.__doc__:
    --- End diff --
    
    This is a good idea, but can you please do it in a separate PR?  This is a broad change, so separating it out would be helpful (in case of conflicts, etc.).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165004850
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-200136101
  
    **[Test build #53872 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53872/consoleFull)** for PR 10242 at commit [`b9b08af`](https://github.com/apache/spark/commit/b9b08af5ee66a55ea1fa7cddc36d690e60ab2fc3).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60467436
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -21,8 +21,9 @@
     from pyspark.ml.param.shared import *
     from pyspark.mllib.common import inherit_doc
     
    -__all__ = ['BisectingKMeans', 'BisectingKMeansModel',
    -           'KMeans', 'KMeansModel']
    +__all__ = ['KMeans', 'KMeansModel',
    --- End diff --
    
    No need to reorder these


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57451116
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    +                              "Output column with estimates of the topic mixture distribution for "
    +                              "each document (often called \"theta\" in the literature). Returns "
    +                              "a vector of zeros for an empty document.")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", k=10, \
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51, \
    +                 subsamplingRate=0.05, optimizeDocConcentration=True, \
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        super(LDA, self).__init__()
    +        self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.LDA", self.uid)
    +        self._setDefault(k=10, optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                         subsamplingRate=0.05, optimizeDocConcentration=True,
    +                         checkpointInterval=10, maxIter=20)
    +        kwargs = self.__init__._input_kwargs
    +        self.setParams(**kwargs)
    +
    +    def _create_model(self, java_model):
    +        if self.getOptimizer() == "em":
    +            return DistributedLDAModel(java_model)
    +        else:
    +            return LocalLDAModel(java_model)
    +
    +    @keyword_only
    +    @since("2.0.0")
    +    def setParams(self, featuresCol="features", k=10,
    +                  optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                  subsamplingRate=0.05, optimizeDocConcentration=True,
    +                  checkpointInterval=10, maxIter=20, seed=None):
    --- End diff --
    
    ```docConcentration, topicConcentration, topicDistributionCol``` should be arguments of ```setParams```.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#issuecomment-214918048
  
    **[Test build #57050 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/57050/consoleFull)** for PR 10242 at commit [`2b2bafe`](https://github.com/apache/spark/commit/2b2bafe28e9da16b722f1bc7161620e20fdce1cf).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-196761704
  
    **[Test build #53188 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53188/consoleFull)** for PR 10242 at commit [`4ab456c`](https://github.com/apache/spark/commit/4ab456c3af9985afdd1a9d218a40b3fb61b476f1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#issuecomment-214918542
  
    OK thanks! I'll update it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54664922
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,317 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "LDA optimizer, Currently 'em', 'online' are supported.")
    +    learningOffSet = Param(Params._dummy(), "learningOffSet",
    +                           "learning parameter that downweights early iterations")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration \
    +                                     will be optimized during training")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        :param featureCol:          feature column name in the dataframe
    +        :param k:                   Number of clusters you want
    +        :param optimizer:           LDAOptimizer used to perform the actual calculation.
    +            Currently "em", "online" are supported. Default to "em".
    +
    +        :param learningOffset:      A (positive) learning parameter that downweights early
    +            iterations. Larger values make early iterations count less. This is called "tau0"
    +            in the Online LDA paper (Hoffman et al., 2010) Default: 1024.0, following Hoffman et al.
    +        :param learningDecay:       Learning rate, set as an exponential decay rate. This should
    +            be between (0.5, 1.0] to guarantee asymptotic convergence. This is called "kappa" in
    +            the Online LDA paper (Hoffman et al., 2010). Default: 0.51, based on Hoffman et al.
    +        :param subsamplingRate:     Fraction of the corpus to be sampled and used in each iteration
    +            of mini-batch gradient descent, in range (0, 1]. Note that this should be adjusted in
    +            synch with [[LDA.maxIter]] so the entire corpus is used.  Specifically, set both so that
    +            maxIterations * miniBatchFraction >= 1. Default:0.05
    +        :param optimizeDocConcentration:       Indicates whether the docConcentration (Dirichlet
    +            parameter for document-topic distribution) will be optimized during training. Setting
    +            this to true will make the model more expressive and fit the training data better.
    +            Default: false
    +        :param checkpointInterval:  Param for set checkpoint interval (>= 1) or disable
    +            checkpoint (-1). E.g. 10 means that the cache will get checkpointed every 10 iterations.
    +        :param maxIter:             Number of iterations. Default to 20
    +        :param seed:                Param for random seed
    --- End diff --
    
    The problem is some parameters are not declared as Param (no get/set method for them). I am not sure the best practise for them. Seems other algorithms are the same. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-190089437
  
    **[Test build #52180 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52180/consoleFull)** for PR 10242 at commit [`b27c275`](https://github.com/apache/spark/commit/b27c2752e25099914fe5536ce2ff250e2f9037e6).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60470030
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    --- End diff --
    
    Double-brackets ```[[...]]``` are for Scala only.  Here, use ```:py:attr:`LDA.docConcentration` ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165020626
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK][ML] Python API for ml.c...

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

    https://github.com/apache/spark/pull/10242#issuecomment-214954035
  
    Thanks, I did the rebase and updated it.  It's in this new PR: [https://github.com/apache/spark/pull/12723]
    
    Could you please close this issue, and if you have time take a look at the new PR?  Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by yanboliang <gi...@git.apache.org>.
Github user yanboliang commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-191147310
  
    @zjffdu Thanks for working on this issue. Then mainly problem of the current PR is that you should check the docs of params and methods, to ensure it is completely consistent with Scala one. We usually directly copy the Scala API doc to Python, because these docs are used to generate Python API docs which is important to PySpark users. Any problems, please feel free to let me know.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165004716
  
    **[Test build #47790 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47790/consoleFull)** for PR 10242 at commit [`ebf5e35`](https://github.com/apache/spark/commit/ebf5e35d3eda97c279c2a57791e6878cc38b054f).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163858445
  
    **[Test build #47573 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47573/consoleFull)** for PR 10242 at commit [`9c2bf31`](https://github.com/apache/spark/commit/9c2bf314a24325211b690531211c32071199c5ef).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class LDAModel(JavaModel):`\n  * `class DistributedLDAModel(LDAModel):`\n  * `class LocalLDAModel(LDAModel):`\n  * `class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):`\n


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r54696423
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -291,6 +292,284 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    --- End diff --
    
    The Python API doc should consistent with Scala one, should also include the ```WARNING``` to tell users that this API may cause collecting a large amount of data to the driver.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-163854051
  
    **[Test build #47573 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47573/consoleFull)** for PR 10242 at commit [`9c2bf31`](https://github.com/apache/spark/commit/9c2bf314a24325211b690531211c32071199c5ef).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-207572654
  
    Ping!  It will be great to get this into 2.0.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r60471273
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -310,6 +311,374 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    --- End diff --
    
    It looks like this is an older copy of the Scala doc. Can you please update it based on master?  The current master's doc has some more details.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

Posted by zjffdu <gi...@git.apache.org>.
Github user zjffdu commented on the pull request:

    https://github.com/apache/spark/pull/10242#issuecomment-164986660
  
    @yanboliang Push another commit to address the comments. BTW, for the unit test, I will get different from if I use python2.7, is it expected ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-165017494
  
    **[Test build #47797 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47797/consoleFull)** for PR 10242 at commit [`bef9c91`](https://github.com/apache/spark/commit/bef9c9148ba3befa9b716e48fb2af916e8d15e20).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r47620780
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -167,6 +167,200 @@ def getInitSteps(self):
             return self.getOrDefault(self.initSteps)
     
     
    +class LDAModel(JavaModel):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +    """
    +
    +    @since("1.7.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("1.7.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("1.7.0")
    +    def topicsMatrix(self):
    +        """Inferred topics, where each topic is represented by a distribution over terms."""
    +        return self._call_java("topicsMatrix")
    +
    +    @since("1.7.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus."""
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("1.7.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)"""
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("1.7.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """Return the topics described by weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +            (default: vocabulary size)
    +        :return: Array over topics. Each topic is represented as a pair of matching arrays:
    +            (term indices, term weights in topic).
    +            Each topic's terms are sorted in order of decreasing weight.
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 1.7.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """ A clustering model derived from the LDA method.
    +
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +
    +    .. versionadded:: 1.7.0
    +    """
    +
    +    # a placeholder to make it appear in the generated doc
    +    k = Param(Params._dummy(), "k", "number of clusters to create")
    +    optimizer = Param(Params._dummy(), "optimizer", "LDA optimizer")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=2,
    +                 optimizer="online", learningOffset=5, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    +        """
    +        __init__(self, featuresCol="features", predictionCol="prediction", k=2, \
    --- End diff --
    
    documents should consistent with function definition.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-164991936
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#issuecomment-212274009
  
    **[Test build #56323 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/56323/consoleFull)** for PR 10242 at commit [`372d5a5`](https://github.com/apache/spark/commit/372d5a5ff5effd63bb103a784f3a1872b49ff6e5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-11940][PYSPARK] Python API for ml.clust...

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

    https://github.com/apache/spark/pull/10242#discussion_r57834098
  
    --- Diff: python/pyspark/ml/clustering.py ---
    @@ -292,6 +293,376 @@ def _create_model(self, java_model):
             return BisectingKMeansModel(java_model)
     
     
    +class LDAModel(JavaModel):
    +    """
    +    A clustering model derived from the LDA method.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    @since("2.0.0")
    +    def isDistributed(self):
    +        """Indicates whether this instance is of type DistributedLDAModel"""
    +        return self._call_java("isDistributed")
    +
    +    @since("2.0.0")
    +    def vocabSize(self):
    +        """Vocabulary size (number of terms or terms in the vocabulary)"""
    +        return self._call_java("vocabSize")
    +
    +    @since("2.0.0")
    +    def topicsMatrix(self):
    +        """ Inferred topics, where each topic is represented by a distribution over terms.
    +        This is a matrix of size vocabSize x k, where each column is a topic.
    +        No guarantees are given about the ordering of the topics.
    +
    +        WARNING: If this model is actually a [[DistributedLDAModel]] instance produced by
    +        the Expectation-Maximization ("em") [[optimizer]], then this method could involve
    +        collecting a large amount of data to the driver (on the order of vocabSize x k).
    +        """
    +        return self._call_java("topicsMatrix")
    +
    +    @since("2.0.0")
    +    def logLikelihood(self, dataset):
    +        """Calculates a lower bound on the log likelihood of the entire corpus.
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logLikelihood", dataset)
    +
    +    @since("2.0.0")
    +    def logPerplexity(self, dataset):
    +        """Calculate an upper bound bound on perplexity.  (Lower is better.)
    +        See Equation (16) in the Online LDA paper (Hoffman et al., 2010).
    +
    +        WARNING: If this model is an instance of [[DistributedLDAModel]] (produced when
    +        [[optimizer]] is set to "em"), this involves collecting a large [[topicsMatrix]] to the
    +        driver. This implementation may be changed in the future.
    +        """
    +        return self._call_java("logPerplexity", dataset)
    +
    +    @since("2.0.0")
    +    def describeTopics(self, maxTermsPerTopic=10):
    +        """ Return the topics described by their top-weighted terms.
    +
    +        WARNING: If vocabSize and k are large, this can return a large object!
    +
    +        :param maxTermsPerTopic: Maximum number of terms to collect for each topic.
    +               Default value of 10.
    +        :return: Local DataFrame with one topic per Row, with columns:
    +                - "topic": IntegerType: topic index
    +                - "termIndices": ArrayType(IntegerType): term indices, sorted in order of decreasing
    +                             term importance
    +                - "termWeights": ArrayType(DoubleType): corresponding sorted term weights
    +        """
    +        return self._call_java("describeTopics", maxTermsPerTopic)
    +
    +    @since("2.0.0")
    +    def estimatedDocConcentration(self):
    +        """Value for [[docConcentration]] estimated from data.
    +        If Online LDA was used and [[optimizeDocConcentration]] was set to false,
    +        then this returns the fixed (given) value for the [[docConcentration]] parameter.
    +        """
    +        return self._call_java("estimatedDocConcentration")
    +
    +
    +class DistributedLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    def toLocal(self):
    +        return self._call_java("toLocal")
    +
    +
    +class LocalLDAModel(LDAModel):
    +    """
    +    Model fitted by LDA.
    +
    +    .. versionadded:: 2.0.0
    +    """
    +    pass
    +
    +
    +class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInterval):
    +    """
    +    Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
    +    Terminology
    +    - "word" = "term": an element of the vocabulary
    +    - "token": instance of a term appearing in a document
    +    - "topic": multinomial distribution over words representing some concept
    +    References:
    +    - Original LDA paper (journal version):
    +    Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
    +
    +    >>> from pyspark.mllib.linalg import Vectors, SparseVector
    +    >>> from pyspark.ml.clustering import LDA
    +    >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], \
    +        [2, SparseVector(2, {0: 1.0})],], ["id", "features"])
    +    >>> lda = LDA(k=2, seed=1, optimizer="em")
    +    >>> model = lda.fit(df)
    +    >>> model.isDistributed()
    +    True
    +    >>> localModel = model.toLocal()
    +    >>> localModel.isDistributed()
    +    False
    +    >>> model.vocabSize()
    +    2
    +    >>> model.describeTopics().show()
    +    +-----+-----------+--------------------+
    +    |topic|termIndices|         termWeights|
    +    +-----+-----------+--------------------+
    +    |    0|     [1, 0]|[0.50401530077160...|
    +    |    1|     [0, 1]|[0.50401530077160...|
    +    +-----+-----------+--------------------+
    +    ...
    +    >>> model.topicsMatrix()
    +    DenseMatrix(2, 2, [0.496, 0.504, 0.504, 0.496], 0)
    +    >>> model.estimatedDocConcentration()
    +    DenseVector([26.0, 26.0])
    +
    +    .. versionadded:: 2.0.0
    +    """
    +
    +    k = Param(Params._dummy(), "k", "number of topics (clusters) to infer")
    +    optimizer = Param(Params._dummy(), "optimizer",
    +                      "Optimizer or inference algorithm used to estimate the LDA model.  "
    +                      "Supported: online, em")
    +    learningOffset = Param(Params._dummy(), "learningOffset",
    +                           "A (positive) learning parameter that downweights early iterations."
    +                           " Larger values make early iterations count less")
    +    learningDecay = Param(Params._dummy(), "learningDecay", "Learning rate, set as an"
    +                          "exponential decay rate. This should be between (0.5, 1.0] to "
    +                          "guarantee asymptotic convergence.")
    +    subsamplingRate = Param(Params._dummy(), "subsamplingRate",
    +                            "Fraction of the corpus to be sampled and used in each iteration "
    +                            "of mini-batch gradient descent, in range (0, 1].")
    +    optimizeDocConcentration = Param(Params._dummy(), "optimizeDocConcentration",
    +                                     "Indicates whether the docConcentration (Dirichlet parameter "
    +                                     "for document-topic distribution) will be optimized during "
    +                                     "training.")
    +    docConcentration = Param(Params._dummy(), "docConcentration",
    +                             "Concentration parameter (commonly named \"alpha\") for the "
    +                             "prior placed on documents' distributions over topics (\"theta\").")
    +    topicConcentration = Param(Params._dummy(), "topicConcentration",
    +                               "Concentration parameter (commonly named \"beta\" or \"eta\") for "
    +                               "the prior placed on topic' distributions over terms.")
    +    topicDistribution = Param(Params._dummy(), "topicDistribution",
    +                              "Output column with estimates of the topic mixture distribution for "
    +                              "each document (often called \"theta\" in the literature). Returns "
    +                              "a vector of zeros for an empty document.")
    +
    +    @keyword_only
    +    def __init__(self, featuresCol="features", k=10,
    +                 optimizer="online", learningOffset=1024.0, learningDecay=0.51,
    +                 subsamplingRate=0.05, optimizeDocConcentration=True,
    +                 checkpointInterval=10, maxIter=20, seed=None):
    --- End diff --
    
    These 3 parameters are not set to default value in scala LDA, so I also didn't include them here.
    
    ```
      setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10,
        learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05,
        optimizeDocConcentration -> true)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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