aboutsummaryrefslogtreecommitdiff
path: root/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala
blob: b5a764b5863f1f893a577fc54e6990a37ef63b3b (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.ml.clustering

import org.apache.hadoop.fs.Path

import org.apache.spark.annotation.{DeveloperApi, Experimental, Since}
import org.apache.spark.internal.Logging
import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT}
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed}
import org.apache.spark.ml.util._
import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedLDAModel,
  EMLDAOptimizer => OldEMLDAOptimizer, LDA => OldLDA, LDAModel => OldLDAModel,
  LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel,
  OnlineLDAOptimizer => OldOnlineLDAOptimizer}
import org.apache.spark.mllib.impl.PeriodicCheckpointer
import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Vector => OldVector,
  Vectors => OldVectors}
import org.apache.spark.mllib.linalg.MatrixImplicits._
import org.apache.spark.mllib.linalg.VectorImplicits._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession}
import org.apache.spark.sql.functions.{col, monotonically_increasing_id, udf}
import org.apache.spark.sql.types.StructType


private[clustering] trait LDAParams extends Params with HasFeaturesCol with HasMaxIter
  with HasSeed with HasCheckpointInterval {

  /**
   * Param for the number of topics (clusters) to infer. Must be > 1. Default: 10.
   *
   * @group param
   */
  @Since("1.6.0")
  final val k = new IntParam(this, "k", "The number of topics (clusters) to infer. " +
    "Must be > 1.", ParamValidators.gt(1))

  /** @group getParam */
  @Since("1.6.0")
  def getK: Int = $(k)

  /**
   * Concentration parameter (commonly named "alpha") for the prior placed on documents'
   * distributions over topics ("theta").
   *
   * This is the parameter to a Dirichlet distribution, where larger values mean more smoothing
   * (more regularization).
   *
   * If not set by the user, then docConcentration is set automatically. If set to
   * singleton vector [alpha], then alpha is replicated to a vector of length k in fitting.
   * Otherwise, the [[docConcentration]] vector must be length k.
   * (default = automatic)
   *
   * Optimizer-specific parameter settings:
   *  - EM
   *     - Currently only supports symmetric distributions, so all values in the vector should be
   *       the same.
   *     - Values should be > 1.0
   *     - default = uniformly (50 / k) + 1, where 50/k is common in LDA libraries and +1 follows
   *       from Asuncion et al. (2009), who recommend a +1 adjustment for EM.
   *  - Online
   *     - Values should be >= 0
   *     - default = uniformly (1.0 / k), following the implementation from
   *       [[https://github.com/Blei-Lab/onlineldavb]].
   * @group param
   */
  @Since("1.6.0")
  final val docConcentration = new DoubleArrayParam(this, "docConcentration",
    "Concentration parameter (commonly named \"alpha\") for the prior placed on documents'" +
      " distributions over topics (\"theta\").", (alpha: Array[Double]) => alpha.forall(_ >= 0.0))

  /** @group getParam */
  @Since("1.6.0")
  def getDocConcentration: Array[Double] = $(docConcentration)

  /** Get docConcentration used by spark.mllib LDA */
  protected def getOldDocConcentration: Vector = {
    if (isSet(docConcentration)) {
      Vectors.dense(getDocConcentration)
    } else {
      Vectors.dense(-1.0)
    }
  }

  /**
   * Concentration parameter (commonly named "beta" or "eta") for the prior placed on topics'
   * distributions over terms.
   *
   * This is the parameter to a symmetric Dirichlet distribution.
   *
   * Note: The topics' distributions over terms are called "beta" in the original LDA paper
   * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009.
   *
   * If not set by the user, then topicConcentration is set automatically.
   *  (default = automatic)
   *
   * Optimizer-specific parameter settings:
   *  - EM
   *     - Value should be > 1.0
   *     - default = 0.1 + 1, where 0.1 gives a small amount of smoothing and +1 follows
   *       Asuncion et al. (2009), who recommend a +1 adjustment for EM.
   *  - Online
   *     - Value should be >= 0
   *     - default = (1.0 / k), following the implementation from
   *       [[https://github.com/Blei-Lab/onlineldavb]].
   * @group param
   */
  @Since("1.6.0")
  final val topicConcentration = new DoubleParam(this, "topicConcentration",
    "Concentration parameter (commonly named \"beta\" or \"eta\") for the prior placed on topic'" +
      " distributions over terms.", ParamValidators.gtEq(0))

  /** @group getParam */
  @Since("1.6.0")
  def getTopicConcentration: Double = $(topicConcentration)

  /** Get topicConcentration used by spark.mllib LDA */
  protected def getOldTopicConcentration: Double = {
    if (isSet(topicConcentration)) {
      getTopicConcentration
    } else {
      -1.0
    }
  }

  /** Supported values for Param [[optimizer]]. */
  @Since("1.6.0")
  final val supportedOptimizers: Array[String] = Array("online", "em")

  /**
   * Optimizer or inference algorithm used to estimate the LDA model.
   * Currently supported (case-insensitive):
   *  - "online": Online Variational Bayes (default)
   *  - "em": Expectation-Maximization
   *
   * For details, see the following papers:
   *  - Online LDA:
   *     Hoffman, Blei and Bach.  "Online Learning for Latent Dirichlet Allocation."
   *     Neural Information Processing Systems, 2010.
   *     [[http://www.cs.columbia.edu/~blei/papers/HoffmanBleiBach2010b.pdf]]
   *  - EM:
   *     Asuncion et al.  "On Smoothing and Inference for Topic Models."
   *     Uncertainty in Artificial Intelligence, 2009.
   *     [[http://arxiv.org/pdf/1205.2662.pdf]]
   *
   * @group param
   */
  @Since("1.6.0")
  final val optimizer = new Param[String](this, "optimizer", "Optimizer or inference" +
    " algorithm used to estimate the LDA model. Supported: " + supportedOptimizers.mkString(", "),
    (o: String) => ParamValidators.inArray(supportedOptimizers).apply(o.toLowerCase))

  /** @group getParam */
  @Since("1.6.0")
  def getOptimizer: String = $(optimizer)

  /**
   * 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.
   *
   * This uses a variational approximation following Hoffman et al. (2010), where the approximate
   * distribution is called "gamma."  Technically, this method returns this approximation "gamma"
   * for each document.
   *
   * @group param
   */
  @Since("1.6.0")
  final val topicDistributionCol = new Param[String](this, "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.")

  setDefault(topicDistributionCol -> "topicDistribution")

  /** @group getParam */
  @Since("1.6.0")
  def getTopicDistributionCol: String = $(topicDistributionCol)

  /**
   * For Online optimizer only: [[optimizer]] = "online".
   *
   * 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, following Hoffman et al.
   *
   * @group expertParam
   */
  @Since("1.6.0")
  final val learningOffset = new DoubleParam(this, "learningOffset", "(For online optimizer)" +
    " A (positive) learning parameter that downweights early iterations. Larger values make early" +
    " iterations count less.",
    ParamValidators.gt(0))

  /** @group expertGetParam */
  @Since("1.6.0")
  def getLearningOffset: Double = $(learningOffset)

  /**
   * For Online optimizer only: [[optimizer]] = "online".
   *
   * 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.
   *
   * @group expertParam
   */
  @Since("1.6.0")
  final val learningDecay = new DoubleParam(this, "learningDecay", "(For online optimizer)" +
    " Learning rate, set as an exponential decay rate. This should be between (0.5, 1.0] to" +
    " guarantee asymptotic convergence.", ParamValidators.gt(0))

  /** @group expertGetParam */
  @Since("1.6.0")
  def getLearningDecay: Double = $(learningDecay)

  /**
   * For Online optimizer only: [[optimizer]] = "online".
   *
   * 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.
   *
   * Note: This is the same as the `miniBatchFraction` parameter in
   *       [[org.apache.spark.mllib.clustering.OnlineLDAOptimizer]].
   *
   * Default: 0.05, i.e., 5% of total documents.
   *
   * @group param
   */
  @Since("1.6.0")
  final val subsamplingRate = new DoubleParam(this, "subsamplingRate", "(For online optimizer)" +
    " Fraction of the corpus to be sampled and used in each iteration of mini-batch" +
    " gradient descent, in range (0, 1].",
    ParamValidators.inRange(0.0, 1.0, lowerInclusive = false, upperInclusive = true))

  /** @group getParam */
  @Since("1.6.0")
  def getSubsamplingRate: Double = $(subsamplingRate)

  /**
   * For Online optimizer only (currently): [[optimizer]] = "online".
   *
   * 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
   *
   * @group expertParam
   */
  @Since("1.6.0")
  final val optimizeDocConcentration = new BooleanParam(this, "optimizeDocConcentration",
    "(For online optimizer only, currently) Indicates whether the docConcentration" +
      " (Dirichlet parameter for document-topic distribution) will be optimized during training.")

  /** @group expertGetParam */
  @Since("1.6.0")
  def getOptimizeDocConcentration: Boolean = $(optimizeDocConcentration)

  /**
   * For EM optimizer only: [[optimizer]] = "em".
   *
   * If using checkpointing, this indicates whether to keep the last
   * checkpoint. If false, then the checkpoint will be deleted. Deleting the checkpoint can
   * cause failures if a data partition is lost, so set this bit with care.
   * Note that checkpoints will be cleaned up via reference counting, regardless.
   *
   * See [[DistributedLDAModel.getCheckpointFiles]] for getting remaining checkpoints and
   * [[DistributedLDAModel.deleteCheckpointFiles]] for removing remaining checkpoints.
   *
   * Default: true
   *
   * @group expertParam
   */
  @Since("2.0.0")
  final val keepLastCheckpoint = new BooleanParam(this, "keepLastCheckpoint",
    "(For EM optimizer) If using checkpointing, this indicates whether to keep the last" +
      " checkpoint. If false, then the checkpoint will be deleted. Deleting the checkpoint can" +
      " cause failures if a data partition is lost, so set this bit with care.")

  /** @group expertGetParam */
  @Since("2.0.0")
  def getKeepLastCheckpoint: Boolean = $(keepLastCheckpoint)

  /**
   * Validates and transforms the input schema.
   *
   * @param schema input schema
   * @return output schema
   */
  protected def validateAndTransformSchema(schema: StructType): StructType = {
    if (isSet(docConcentration)) {
      if (getDocConcentration.length != 1) {
        require(getDocConcentration.length == getK, s"LDA docConcentration was of length" +
          s" ${getDocConcentration.length}, but k = $getK.  docConcentration must be an array of" +
          s" length either 1 (scalar) or k (num topics).")
      }
      getOptimizer match {
        case "online" =>
          require(getDocConcentration.forall(_ >= 0),
            "For Online LDA optimizer, docConcentration values must be >= 0.  Found values: " +
              getDocConcentration.mkString(","))
        case "em" =>
          require(getDocConcentration.forall(_ >= 0),
            "For EM optimizer, docConcentration values must be >= 1.  Found values: " +
              getDocConcentration.mkString(","))
      }
    }
    if (isSet(topicConcentration)) {
      getOptimizer match {
        case "online" =>
          require(getTopicConcentration >= 0, s"For Online LDA optimizer, topicConcentration" +
            s" must be >= 0.  Found value: $getTopicConcentration")
        case "em" =>
          require(getTopicConcentration >= 0, s"For EM optimizer, topicConcentration" +
            s" must be >= 1.  Found value: $getTopicConcentration")
      }
    }
    SchemaUtils.checkColumnType(schema, $(featuresCol), new VectorUDT)
    SchemaUtils.appendColumn(schema, $(topicDistributionCol), new VectorUDT)
  }

  private[clustering] def getOldOptimizer: OldLDAOptimizer = getOptimizer match {
    case "online" =>
      new OldOnlineLDAOptimizer()
        .setTau0($(learningOffset))
        .setKappa($(learningDecay))
        .setMiniBatchFraction($(subsamplingRate))
        .setOptimizeDocConcentration($(optimizeDocConcentration))
    case "em" =>
      new OldEMLDAOptimizer()
        .setKeepLastCheckpoint($(keepLastCheckpoint))
  }
}


/**
 * :: Experimental ::
 * Model fitted by [[LDA]].
 *
 * @param vocabSize  Vocabulary size (number of terms or words in the vocabulary)
 * @param sparkSession  Used to construct local DataFrames for returning query results
 */
@Since("1.6.0")
@Experimental
sealed abstract class LDAModel private[ml] (
    @Since("1.6.0") override val uid: String,
    @Since("1.6.0") val vocabSize: Int,
    @Since("1.6.0") @transient private[ml] val sparkSession: SparkSession)
  extends Model[LDAModel] with LDAParams with Logging with MLWritable {

  // NOTE to developers:
  //  This abstraction should contain all important functionality for basic LDA usage.
  //  Specializations of this class can contain expert-only functionality.

  /**
   * Underlying spark.mllib model.
   * If this model was produced by Online LDA, then this is the only model representation.
   * If this model was produced by EM, then this local representation may be built lazily.
   */
  @Since("1.6.0")
  protected def oldLocalModel: OldLocalLDAModel

  /** Returns underlying spark.mllib model, which may be local or distributed */
  @Since("1.6.0")
  protected def getModel: OldLDAModel

  private[ml] def getEffectiveDocConcentration: Array[Double] = getModel.docConcentration.toArray

  private[ml] def getEffectiveTopicConcentration: Double = getModel.topicConcentration

  /**
   * The features for LDA should be a [[Vector]] representing the word counts in a document.
   * The vector should be of length vocabSize, with counts for each term (word).
   *
   * @group setParam
   */
  @Since("1.6.0")
  def setFeaturesCol(value: String): this.type = set(featuresCol, value)

  /** @group setParam */
  @Since("1.6.0")
  def setSeed(value: Long): this.type = set(seed, value)

  /**
   * Transforms the input dataset.
   *
   * 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.
   */
  @Since("2.0.0")
  override def transform(dataset: Dataset[_]): DataFrame = {
    if ($(topicDistributionCol).nonEmpty) {

      // TODO: Make the transformer natively in ml framework to avoid extra conversion.
      val transformer = oldLocalModel.getTopicDistributionMethod(sparkSession.sparkContext)

      val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML }
      dataset.withColumn($(topicDistributionCol), t(col($(featuresCol)))).toDF
    } else {
      logWarning("LDAModel.transform was called without any output columns. Set an output column" +
        " such as topicDistributionCol to produce results.")
      dataset.toDF
    }
  }

  @Since("1.6.0")
  override def transformSchema(schema: StructType): StructType = {
    validateAndTransformSchema(schema)
  }

  /**
   * 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.
   */
  @Since("2.0.0")
  def estimatedDocConcentration: Vector = getModel.docConcentration

  /**
   * 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).
   */
  @Since("2.0.0")
  def topicsMatrix: Matrix = oldLocalModel.topicsMatrix.asML

  /** Indicates whether this instance is of type [[DistributedLDAModel]] */
  @Since("1.6.0")
  def isDistributed: Boolean

  /**
   * 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.
   *
   * @param dataset  test corpus to use for calculating log likelihood
   * @return variational lower bound on the log likelihood of the entire corpus
   */
  @Since("2.0.0")
  def logLikelihood(dataset: Dataset[_]): Double = {
    val oldDataset = LDA.getOldDataset(dataset, $(featuresCol))
    oldLocalModel.logLikelihood(oldDataset)
  }

  /**
   * 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.
   *
   * @param dataset test corpus to use for calculating perplexity
   * @return Variational upper bound on log perplexity per token.
   */
  @Since("2.0.0")
  def logPerplexity(dataset: Dataset[_]): Double = {
    val oldDataset = LDA.getOldDataset(dataset, $(featuresCol))
    oldLocalModel.logPerplexity(oldDataset)
  }

  /**
   * Return the topics described by their top-weighted terms.
   *
   * @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
   */
  @Since("1.6.0")
  def describeTopics(maxTermsPerTopic: Int): DataFrame = {
    val topics = getModel.describeTopics(maxTermsPerTopic).zipWithIndex.map {
      case ((termIndices, termWeights), topic) =>
        (topic, termIndices.toSeq, termWeights.toSeq)
    }
    sparkSession.createDataFrame(topics).toDF("topic", "termIndices", "termWeights")
  }

  @Since("1.6.0")
  def describeTopics(): DataFrame = describeTopics(10)
}


/**
 * :: Experimental ::
 *
 * Local (non-distributed) model fitted by [[LDA]].
 *
 * This model stores the inferred topics only; it does not store info about the training dataset.
 */
@Since("1.6.0")
@Experimental
class LocalLDAModel private[ml] (
    uid: String,
    vocabSize: Int,
    @Since("1.6.0") override protected val oldLocalModel: OldLocalLDAModel,
    sparkSession: SparkSession)
  extends LDAModel(uid, vocabSize, sparkSession) {

  @Since("1.6.0")
  override def copy(extra: ParamMap): LocalLDAModel = {
    val copied = new LocalLDAModel(uid, vocabSize, oldLocalModel, sparkSession)
    copyValues(copied, extra).setParent(parent).asInstanceOf[LocalLDAModel]
  }

  override protected def getModel: OldLDAModel = oldLocalModel

  @Since("1.6.0")
  override def isDistributed: Boolean = false

  @Since("1.6.0")
  override def write: MLWriter = new LocalLDAModel.LocalLDAModelWriter(this)
}


@Since("1.6.0")
object LocalLDAModel extends MLReadable[LocalLDAModel] {

  private[LocalLDAModel]
  class LocalLDAModelWriter(instance: LocalLDAModel) extends MLWriter {

    private case class Data(
        vocabSize: Int,
        topicsMatrix: Matrix,
        docConcentration: Vector,
        topicConcentration: Double,
        gammaShape: Double)

    override protected def saveImpl(path: String): Unit = {
      DefaultParamsWriter.saveMetadata(instance, path, sc)
      val oldModel = instance.oldLocalModel
      val data = Data(instance.vocabSize, oldModel.topicsMatrix, oldModel.docConcentration,
        oldModel.topicConcentration, oldModel.gammaShape)
      val dataPath = new Path(path, "data").toString
      sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath)
    }
  }

  private class LocalLDAModelReader extends MLReader[LocalLDAModel] {

    private val className = classOf[LocalLDAModel].getName

    override def load(path: String): LocalLDAModel = {
      val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
      val dataPath = new Path(path, "data").toString
      val data = sparkSession.read.parquet(dataPath)
        .select("vocabSize", "topicsMatrix", "docConcentration", "topicConcentration",
          "gammaShape")
        .head()
      val vocabSize = data.getAs[Int](0)
      val topicsMatrix = data.getAs[Matrix](1)
      val docConcentration = data.getAs[Vector](2)
      val topicConcentration = data.getAs[Double](3)
      val gammaShape = data.getAs[Double](4)
      val oldModel = new OldLocalLDAModel(topicsMatrix, docConcentration, topicConcentration,
        gammaShape)
      val model = new LocalLDAModel(metadata.uid, vocabSize, oldModel, sparkSession)
      DefaultParamsReader.getAndSetParams(model, metadata)
      model
    }
  }

  @Since("1.6.0")
  override def read: MLReader[LocalLDAModel] = new LocalLDAModelReader

  @Since("1.6.0")
  override def load(path: String): LocalLDAModel = super.load(path)
}


/**
 * :: Experimental ::
 *
 * Distributed model fitted by [[LDA]].
 * This type of model is currently only produced by Expectation-Maximization (EM).
 *
 * This model stores the inferred topics, the full training dataset, and the topic distribution
 * for each training document.
 *
 * @param oldLocalModelOption  Used to implement [[oldLocalModel]] as a lazy val, but keeping
 *                             [[copy()]] cheap.
 */
@Since("1.6.0")
@Experimental
class DistributedLDAModel private[ml] (
    uid: String,
    vocabSize: Int,
    private val oldDistributedModel: OldDistributedLDAModel,
    sparkSession: SparkSession,
    private var oldLocalModelOption: Option[OldLocalLDAModel])
  extends LDAModel(uid, vocabSize, sparkSession) {

  override protected def oldLocalModel: OldLocalLDAModel = {
    if (oldLocalModelOption.isEmpty) {
      oldLocalModelOption = Some(oldDistributedModel.toLocal)
    }
    oldLocalModelOption.get
  }

  override protected def getModel: OldLDAModel = oldDistributedModel

  /**
   * Convert this distributed model to a local representation.  This discards info about the
   * training dataset.
   *
   * WARNING: This involves collecting a large [[topicsMatrix]] to the driver.
   */
  @Since("1.6.0")
  def toLocal: LocalLDAModel = new LocalLDAModel(uid, vocabSize, oldLocalModel, sparkSession)

  @Since("1.6.0")
  override def copy(extra: ParamMap): DistributedLDAModel = {
    val copied = new DistributedLDAModel(
      uid, vocabSize, oldDistributedModel, sparkSession, oldLocalModelOption)
    copyValues(copied, extra).setParent(parent)
    copied
  }

  @Since("1.6.0")
  override def isDistributed: Boolean = true

  /**
   * Log likelihood of the observed tokens in the training set,
   * given the current parameter estimates:
   *  log P(docs | topics, topic distributions for docs, Dirichlet hyperparameters)
   *
   * Notes:
   *  - This excludes the prior; for that, use [[logPrior]].
   *  - Even with [[logPrior]], this is NOT the same as the data log likelihood given the
   *    hyperparameters.
   *  - This is computed from the topic distributions computed during training. If you call
   *    [[logLikelihood()]] on the same training dataset, the topic distributions will be computed
   *    again, possibly giving different results.
   */
  @Since("1.6.0")
  lazy val trainingLogLikelihood: Double = oldDistributedModel.logLikelihood

  /**
   * Log probability of the current parameter estimate:
   * log P(topics, topic distributions for docs | Dirichlet hyperparameters)
   */
  @Since("1.6.0")
  lazy val logPrior: Double = oldDistributedModel.logPrior

  private var _checkpointFiles: Array[String] = oldDistributedModel.checkpointFiles

  /**
   * :: DeveloperApi ::
   *
   * If using checkpointing and [[LDA.keepLastCheckpoint]] is set to true, then there may be
   * saved checkpoint files.  This method is provided so that users can manage those files.
   *
   * Note that removing the checkpoints can cause failures if a partition is lost and is needed
   * by certain [[DistributedLDAModel]] methods.  Reference counting will clean up the checkpoints
   * when this model and derivative data go out of scope.
   *
   * @return  Checkpoint files from training
   */
  @DeveloperApi
  @Since("2.0.0")
  def getCheckpointFiles: Array[String] = _checkpointFiles

  /**
   * :: DeveloperApi ::
   *
   * Remove any remaining checkpoint files from training.
   *
   * @see [[getCheckpointFiles]]
   */
  @DeveloperApi
  @Since("2.0.0")
  def deleteCheckpointFiles(): Unit = {
    val hadoopConf = sparkSession.sparkContext.hadoopConfiguration
    _checkpointFiles.foreach(PeriodicCheckpointer.removeCheckpointFile(_, hadoopConf))
    _checkpointFiles = Array.empty[String]
  }

  @Since("1.6.0")
  override def write: MLWriter = new DistributedLDAModel.DistributedWriter(this)
}


@Since("1.6.0")
object DistributedLDAModel extends MLReadable[DistributedLDAModel] {

  private[DistributedLDAModel]
  class DistributedWriter(instance: DistributedLDAModel) extends MLWriter {

    override protected def saveImpl(path: String): Unit = {
      DefaultParamsWriter.saveMetadata(instance, path, sc)
      val modelPath = new Path(path, "oldModel").toString
      instance.oldDistributedModel.save(sc, modelPath)
    }
  }

  private class DistributedLDAModelReader extends MLReader[DistributedLDAModel] {

    private val className = classOf[DistributedLDAModel].getName

    override def load(path: String): DistributedLDAModel = {
      val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
      val modelPath = new Path(path, "oldModel").toString
      val oldModel = OldDistributedLDAModel.load(sc, modelPath)
      val model = new DistributedLDAModel(
        metadata.uid, oldModel.vocabSize, oldModel, sparkSession, None)
      DefaultParamsReader.getAndSetParams(model, metadata)
      model
    }
  }

  @Since("1.6.0")
  override def read: MLReader[DistributedLDAModel] = new DistributedLDAModelReader

  @Since("1.6.0")
  override def load(path: String): DistributedLDAModel = super.load(path)
}


/**
 * :: Experimental ::
 *
 * Latent Dirichlet Allocation (LDA), a topic model designed for text documents.
 *
 * Terminology:
 *  - "term" = "word": an element of the vocabulary
 *  - "token": instance of a term appearing in a document
 *  - "topic": multinomial distribution over terms representing some concept
 *  - "document": one piece of text, corresponding to one row in the input data
 *
 * Original LDA paper (journal version):
 *  Blei, Ng, and Jordan.  "Latent Dirichlet Allocation."  JMLR, 2003.
 *
 * Input data (featuresCol):
 *  LDA is given a collection of documents as input data, via the featuresCol parameter.
 *  Each document is specified as a [[Vector]] of length vocabSize, where each entry is the
 *  count for the corresponding term (word) in the document.  Feature transformers such as
 *  [[org.apache.spark.ml.feature.Tokenizer]] and [[org.apache.spark.ml.feature.CountVectorizer]]
 *  can be useful for converting text to word count vectors.
 *
 * @see [[http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation Latent Dirichlet allocation
 *       (Wikipedia)]]
 */
@Since("1.6.0")
@Experimental
class LDA @Since("1.6.0") (
    @Since("1.6.0") override val uid: String)
  extends Estimator[LDAModel] with LDAParams with DefaultParamsWritable {

  @Since("1.6.0")
  def this() = this(Identifiable.randomUID("lda"))

  setDefault(maxIter -> 20, k -> 10, optimizer -> "online", checkpointInterval -> 10,
    learningOffset -> 1024, learningDecay -> 0.51, subsamplingRate -> 0.05,
    optimizeDocConcentration -> true, keepLastCheckpoint -> true)

  /**
   * The features for LDA should be a [[Vector]] representing the word counts in a document.
   * The vector should be of length vocabSize, with counts for each term (word).
   *
   * @group setParam
   */
  @Since("1.6.0")
  def setFeaturesCol(value: String): this.type = set(featuresCol, value)

  /** @group setParam */
  @Since("1.6.0")
  def setMaxIter(value: Int): this.type = set(maxIter, value)

  /** @group setParam */
  @Since("1.6.0")
  def setSeed(value: Long): this.type = set(seed, value)

  /** @group setParam */
  @Since("1.6.0")
  def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value)

  /** @group setParam */
  @Since("1.6.0")
  def setK(value: Int): this.type = set(k, value)

  /** @group setParam */
  @Since("1.6.0")
  def setDocConcentration(value: Array[Double]): this.type = set(docConcentration, value)

  /** @group setParam */
  @Since("1.6.0")
  def setDocConcentration(value: Double): this.type = set(docConcentration, Array(value))

  /** @group setParam */
  @Since("1.6.0")
  def setTopicConcentration(value: Double): this.type = set(topicConcentration, value)

  /** @group setParam */
  @Since("1.6.0")
  def setOptimizer(value: String): this.type = set(optimizer, value)

  /** @group setParam */
  @Since("1.6.0")
  def setTopicDistributionCol(value: String): this.type = set(topicDistributionCol, value)

  /** @group expertSetParam */
  @Since("1.6.0")
  def setLearningOffset(value: Double): this.type = set(learningOffset, value)

  /** @group expertSetParam */
  @Since("1.6.0")
  def setLearningDecay(value: Double): this.type = set(learningDecay, value)

  /** @group setParam */
  @Since("1.6.0")
  def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value)

  /** @group expertSetParam */
  @Since("1.6.0")
  def setOptimizeDocConcentration(value: Boolean): this.type = set(optimizeDocConcentration, value)

  /** @group expertSetParam */
  @Since("2.0.0")
  def setKeepLastCheckpoint(value: Boolean): this.type = set(keepLastCheckpoint, value)

  @Since("1.6.0")
  override def copy(extra: ParamMap): LDA = defaultCopy(extra)

  @Since("2.0.0")
  override def fit(dataset: Dataset[_]): LDAModel = {
    transformSchema(dataset.schema, logging = true)
    val oldLDA = new OldLDA()
      .setK($(k))
      .setDocConcentration(getOldDocConcentration)
      .setTopicConcentration(getOldTopicConcentration)
      .setMaxIterations($(maxIter))
      .setSeed($(seed))
      .setCheckpointInterval($(checkpointInterval))
      .setOptimizer(getOldOptimizer)
    // TODO: persist here, or in old LDA?
    val oldData = LDA.getOldDataset(dataset, $(featuresCol))
    val oldModel = oldLDA.run(oldData)
    val newModel = oldModel match {
      case m: OldLocalLDAModel =>
        new LocalLDAModel(uid, m.vocabSize, m, dataset.sparkSession)
      case m: OldDistributedLDAModel =>
        new DistributedLDAModel(uid, m.vocabSize, m, dataset.sparkSession, None)
    }
    copyValues(newModel).setParent(this)
  }

  @Since("1.6.0")
  override def transformSchema(schema: StructType): StructType = {
    validateAndTransformSchema(schema)
  }
}

@Since("2.0.0")
object LDA extends DefaultParamsReadable[LDA] {

  /** Get dataset for spark.mllib LDA */
  private[clustering] def getOldDataset(
       dataset: Dataset[_],
       featuresCol: String): RDD[(Long, OldVector)] = {
    dataset
      .withColumn("docId", monotonically_increasing_id())
      .select("docId", featuresCol)
      .rdd
      .map { case Row(docId: Long, features: Vector) =>
        (docId, OldVectors.fromML(features))
      }
  }

  @Since("2.0.0")
  override def load(path: String): LDA = super.load(path)
}