From 0cb718760292fcf015ee1f54eb680e830024bb75 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 16 Dec 2014 15:58:36 -0800 Subject: [PATCH 01/30] Added 3 files from dlwh LDA implementation --- .../SimpleLatentDirichletAllocation.scala | 692 ++++++++++++++++++ mllib/pom.xml | 5 + .../LatentDirichletAllocation.scala | 223 ++++++ 3 files changed, 920 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala new file mode 100644 index 0000000000000..d3c30b292cb23 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala @@ -0,0 +1,692 @@ +/* + * 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.examples.mllib + +import scopt.OptionParser +import org.apache.spark.{SparkContext, SparkConf} +import scala.collection.mutable.ArrayBuffer +import java.text.BreakIterator +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.topicmodel.LatentDirichletAllocation +import org.apache.spark.mllib.topicmodel.LatentDirichletAllocation.Document +import org.apache.spark.mllib.linalg.SparseVector +import org.apache.log4j.{Level, Logger} + +/** + * + * + * @author dlwh + */ +object SimpleLatentDirichletAllocation { + case class Params( + input: Seq[String] = Seq.empty, + numTopics: Int = 20, + wordSmoothing: Double = 0.1, + topicSmoothing: Double = 0.1, + vocabSize: Int = 10000, + minWordCount: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + + val parser = new OptionParser[Params]("SimpleLatentDirichletAllocation") { + head("SimpleLatentDirichletAllocation: an example LDA app for plain text data.") + opt[Int]("numTopics") + .text("number of topics") + .action((x, c) => c.copy(numTopics = x)) + opt[Double]("wordSmoothing") + .text("amount of word smoothing to use") + .action((x, c) => c.copy(wordSmoothing = x)) + opt[Double]("topicSmoothing") + .text(s"amount of topic smoothing to use") + .action((x, c) => c.copy(topicSmoothing = x)) + opt[Int]("vocabSize") + .text(s"number of distinct word types to use, chosen by frequency (after stopword removal)") + .action((x, c) => c.copy(vocabSize = x)) + opt[Int]("minWordCount") + .text(s"minimum number of times a word must appear to be included in vocab.") + .action((x, c) => c.copy(minWordCount = x)) + arg[String]("...") + .text("input paths (directories) to plain text corpora") + .unbounded() + .required() + .action((x, c) => c.copy(input = c.input :+ x)) + } + + val params = parser.parse(args, Params()).getOrElse{parser.showUsageAsError; sys.exit(1)} + + val conf = new SparkConf().setAppName(s"LDA with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val corpus = preprocess(sc, params.input, params.vocabSize, params.minWordCount) + corpus.cache() + + val lda = new LatentDirichletAllocation(params.numTopics, + 100, + params.topicSmoothing, + params.wordSmoothing, + 0) + + for (state <- lda.iterations(corpus)) { + println(state.logLikelihood) + } + } + + def preprocess(sc: SparkContext, + paths: Seq[String], + vocabSize: Int, + minWordCount: Int): RDD[Document] = { + val files = for(p <- paths) yield { + sc.wholeTextFiles(p) + } + + val textRDD = files.reduce( _ ++ _ ) + + val tokenized = textRDD.zipWithIndex.map { case ((name, content), id) => + id -> SimpleTokenizer.getWords(content) + } + + val wordCounts: RDD[(String, Int)] = { + tokenized + .flatMap{ case (_, tokens) => tokens.map(_ -> 1)} + .reduceByKey(_ + _) + .filter(_._2 >= minWordCount) + } + + // word -> id + val vocab = ( + wordCounts + .sortBy(_._2, ascending = false) + .take(vocabSize) + .map(_._1) + .zipWithIndex + .toMap + ) + + val documents = tokenized.map { case (id, toks) => + val counts = breeze.linalg.Counter.countTraversable(toks) + + val indexedCounts = counts.iterator.collect { case (k, v) if vocab.contains(k) => + vocab(k) -> v.toDouble + } + + val sb = org.apache.spark.mllib.linalg.Vectors.sparse(vocab.size, indexedCounts.toSeq) + // I do not know why .sparse doesn't return a SparseVector. + LatentDirichletAllocation.Document(sb.asInstanceOf[SparseVector], id) + } + + documents + } + +} + +object SimpleTokenizer { + + val allWordRegex = "^(\\p{L}|\\p{M})*$".r + + def getWords(text: String): IndexedSeq[String] = { + val words = new ArrayBuffer[String]() + val wb = BreakIterator.getWordInstance + wb.setText(text) + + var current = wb.first() + var end = wb.next() + while (end != BreakIterator.DONE) { + val word: String = text.substring(current, end).toLowerCase + + // remove short words, things that aren't only letters, and stop words + if (allWordRegex.unapplySeq(word).nonEmpty && !stopWords(word) && word.length >= 3) { + words += word + } + + current = end + end = wb.next() + } + words + } + + val stopWords = + """ + |a + |able + |about + |above + |abst + |accordance + |according + |accordingly + |across + |act + |actually + |added + |adj + |affected + |affecting + |affects + |after + |afterwards + |again + |against + |ah + |all + |almost + |alone + |along + |already + |also + |although + |always + |am + |among + |amongst + |an + |and + |announce + |another + |any + |anybody + |anyhow + |anymore + |anyone + |anything + |anyway + |anyways + |anywhere + |apparently + |approximately + |are + |aren + |arent + |arise + |around + |as + |aside + |ask + |asking + |at + |auth + |available + |away + |awfully + |b + |back + |be + |became + |because + |become + |becomes + |becoming + |been + |before + |beforehand + |begin + |beginning + |beginnings + |begins + |behind + |being + |believe + |below + |beside + |besides + |between + |beyond + |biol + |both + |brief + |briefly + |but + |by + |c + |ca + |came + |can + |cannot + |can't + |cause + |causes + |certain + |certainly + |co + |com + |come + |comes + |contain + |containing + |contains + |could + |couldnt + |d + |date + |did + |didn't + |different + |do + |does + |doesn't + |doing + |done + |don't + |down + |downwards + |due + |during + |e + |each + |ed + |edu + |effect + |eg + |eight + |eighty + |either + |else + |elsewhere + |end + |ending + |enough + |especially + |et + |et-al + |etc + |even + |ever + |every + |everybody + |everyone + |everything + |everywhere + |ex + |except + |f + |far + |few + |ff + |fifth + |first + |five + |fix + |followed + |following + |follows + |for + |former + |formerly + |forth + |found + |four + |from + |further + |furthermore + |g + |gave + |get + |gets + |getting + |give + |given + |gives + |giving + |go + |goes + |gone + |got + |gotten + |h + |had + |happens + |hardly + |has + |hasn't + |have + |haven't + |having + |he + |hed + |hence + |her + |here + |hereafter + |hereby + |herein + |heres + |hereupon + |hers + |herself + |hes + |hi + |hid + |him + |himself + |his + |hither + |home + |how + |howbeit + |however + |hundred + |i + |id + |ie + |if + |i'll + |im + |immediate + |immediately + |importance + |important + |in + |inc + |indeed + |index + |information + |instead + |into + |invention + |inward + |is + |isn't + |it + |itd + |it'll + |its + |itself + |i've + |j + |just + |k + |keep keeps + |kept + |kg + |km + |know + |known + |knows + |l + |largely + |last + |lately + |later + |latter + |latterly + |least + |less + |lest + |let + |lets + |like + |liked + |likely + |line + |little + |'ll + |look + |looking + |looks + |ltd + |m + |made + |mainly + |make + |makes + |many + |may + |maybe + |me + |mean + |means + |meantime + |meanwhile + |merely + |mg + |might + |million + |miss + |ml + |more + |moreover + |most + |mostly + |mr + |mrs + |much + |mug + |must + |my + |myself + |n + |na + |name + |namely + |nay + |nd + |near + |nearly + |necessarily + |necessary + |need + |needs + |neither + |never + |nevertheless + |new + |next + |nine + |ninety + |no + |nobody + |non + |none + |nonetheless + |noone + |nor + |normally + |nos + |not + |noted + |nothing + |now + |nowhere + |o + |obtain + |obtained + |obviously + |of + |off + |often + |oh + |ok + |okay + |old + |omitted + |on + |once + |one + |ones + |only + |onto + |or + |ord + |other + |others + |otherwise + |ought + |our + |ours + |ourselves + |out + |outside + |over + |overall + |owing + |own + |p + |page + |pages + |part + |particular + |particularly + |past + |per + |perhaps + |placed + |please + |plus + |poorly + |possible + |possibly + |potentially + |pp + |predominantly + |present + |previously + |primarily + |probably + |promptly + |proud + |provides + |put + |q + |que + |quickly + |quite + |qv + |r + |ran + |rather + |rd + |re + |readily + |really + |recent + |recently + |ref + |refs + |regarding + |regardless + |regards + |related + |relatively + |research + |respectively + |resulted + |resulting + |results + |right + |run + |s + |said + |same + |saw + |say + |saying + |says + |sec + |section + |see + |seeing + |seem + |seemed + |seeming + |seems + |seen + |self + |selves + |sent + |seven + |several + |shall + |she + |shed + |she'll + |shes + |should + |shouldn't + |show + |showed + |shown + |showns + |shows + |significant + |significantly + |similar + |similarly + |since + |six + |slightly + |so + |some + |somebody + |somehow + |someone + |somethan + |something + |sometime + |sometimes + |somewhat + |somewhere + |soon + |sorry + |specifically + |specified + |specify + |specifying + |still + |stop + |strongly + |sub + |substantially + |successfully + |such + |sufficiently + |suggest + |sup + |sure + |t's take taken tell tends + |th than thank thanks thanx + |that that's thats the their + |theirs them themselves then thence + |there there's thereafter thereby therefore + |therein theres thereupon these they + |they'd they'll they're they've think + |third this thorough thoroughly those + |though three through throughout thru + |thus to together too took + |toward towards tried tries truly + |try trying twice two un + |under unfortunately unless unlikely until + |unto up upon us use + |used useful uses using usually + |value various very via viz + |vs want wants was wasn't + |way we we'd we'll we're + |we've welcome well went were + |weren't what what's whatever when + |whence whenever where where's whereafter + |whereas whereby wherein whereupon wherever + |whether which while whither who + |who's whoever whole whom whose + |why will willing wish with + |within without won't wonder would + |wouldn't yes yet you you'd + |you'll you're you've your yours + |yourself yourselves zero + """.stripMargin.split("\\s+").toSet + +} diff --git a/mllib/pom.xml b/mllib/pom.xml index a8cee3d51a780..f1f8c886f949e 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -55,6 +55,11 @@ spark-graphx_${scala.binary.version} ${project.version} + + org.apache.spark + spark-graphx_${scala.binary.version} + ${project.version} + org.jblas jblas diff --git a/mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala b/mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala new file mode 100644 index 0000000000000..24bb398904494 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala @@ -0,0 +1,223 @@ +/* + * 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.mllib.topicmodel + +import org.apache.spark.rdd.RDD +import java.util.Random +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.util.BoundedPriorityQueue +import org.apache.hadoop.fs.shell.Count + + + + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, sum => brzSum, normalize} +import org.apache.spark.mllib.linalg.{DenseVector => SDV, SparseVector => SSV, Vector => SV} + +import org.apache.spark.graphx._ + + +/** + * + * + */ +class LatentDirichletAllocation(var numTopics: Int, + var maxIterations: Int, + var topicSmoothing: Double, + var wordSmoothing: Double, + randomSeed: Long) { + def this(numTopics: Int, maxIterations: Int) = { + this(numTopics, maxIterations, 0.1, 0.1, System.currentTimeMillis()) + } + + def this(numTopics: Int) = this(numTopics, 100) + + + def setNumTopics(k: Int):this.type = {numTopics = k; this} + + def setTopicSmoothing(alpha: Double):this.type = {topicSmoothing = alpha; this} + + def setWordSmoothing(beta: Double):this.type = {wordSmoothing = beta; this} + + import LatentDirichletAllocation._ + + def iterations(docs: RDD[LatentDirichletAllocation.Document]):Iterator[State] = { + val state = initialState(docs, numTopics, wordSmoothing, topicSmoothing, randomSeed) + Iterator.iterate(state)(_.next()).drop(2).take(maxIterations) + } + + def run(docs: RDD[LatentDirichletAllocation.Document]):State = { + import breeze.util.Implicits._ + iterations(docs).last + } +} + +object LatentDirichletAllocation { + case class Document(counts: SSV, id: VertexId) + + private type TopicCounts = BDV[Double] + // Strictly should be an integer, but the algorithm works with Doubles + private type WordCount = Double + + trait State { + def logLikelihood: Double + + def topWords(k: Int):Array[Array[(Double, Int)]] + } + + /** + * + * Has all the information needed to run EM. + * + * The Graph has two kinds of nodes: words and documents. The attr for a word + * is p(w|z). The attr for a document is p(z|doc) + * + * @param graph + * @param numTopics + * @param numWords + * @param topicSmoothing + * @param wordSmoothing + * @param numEStepIters + */ + private case class LearningState(graph: Graph[TopicCounts, Double], + numTopics: Int, + numWords: Int, + topicSmoothing: Double, + wordSmoothing: Double, + numEStepIters: Int = 10) extends State { + + def next() = copy(graph = mStep(eStep(graph))) + + // update p(z|doc) for each doc + private def eStep(graph: Graph[TopicCounts, Double]): Graph[TopicCounts, Double] = { + (0 until numEStepIters).foldLeft(graph) { (graph, _) => + // TODO: we should be able to detect which documents have converged and + // filter them so we don't bother with them for the rest of the estep + val docTopicTotals = updateExpectedCounts(graph, _.srcId) + val alpha = topicSmoothing + val newTotals = docTopicTotals.mapValues(total => normalize(total += alpha, 1)) + graph.outerJoinVertices(newTotals){ (vid, old, newOpt) => newOpt.getOrElse(old)} + } + } + + // update p(w|z) for each word + private def mStep(graph: Graph[TopicCounts, Double]): Graph[TopicCounts, Double] = { + val wordTotals = updateExpectedCounts(graph, _.dstId) + val beta: Double = wordSmoothing + val topicTotals = wordTotals.map(_._2).fold(BDV.zeros[Double](numTopics))(_ + _) + // smooth the totals + topicTotals += (beta * numWords) + + graph.outerJoinVertices(wordTotals)( (vid, old, newOpt) => + newOpt + .map ( counts => (counts += beta) :/= topicTotals) // smooth individual counts; normalize + .getOrElse(old) // keep old p(z|doc) vectors + ) + } + + lazy val logLikelihood = { + graph.triplets.aggregate(0.0)({ (acc, triple) => + val scores = triple.srcAttr :* triple.dstAttr + val logScores = breeze.numerics.log(scores) + scores /= brzSum(scores) + brzSum(scores :*= logScores) * triple.attr + }, _ + _) + } + + // cribbed from jegonzal's implementation + def topWords(k: Int): Array[Array[(Double, Int)]] = { + val nt = numTopics + val nw = numWords + graph.vertices.filter { + case (vid, c) => vid < nw + }.mapPartitions { items => + val queues = Array.fill(nt)(new BoundedPriorityQueue[(Double, Int)](k)) + for ((wordId, factor) <- items) { + var t = 0 + while (t < nt) { + queues(t) += (factor(t) -> wordId.toInt) + t += 1 + } + } + Iterator(queues) + }.reduce { (q1, q2) => + q1.zip(q2).foreach { case (a,b) => a ++= b } + q1 + }.map ( q => q.toArray ) + } + + + } + + + private def updateExpectedCounts(wordCountGraph: Graph[TopicCounts, Double], + sendToWhere: (EdgeTriplet[_, _]) => VertexId) = { + wordCountGraph.mapReduceTriplets[TopicCounts]({ + trip => Iterator(sendToWhere(trip) -> computePTopic(trip)) + }, _ += _) + } + + /** + * Compute bipartite term/doc graph. doc ids are shifted by numWords to maintain uniqueness + * @param docs + * @param numTopics + * @param randomSeed + * @return + */ + private def initialState(docs: RDD[LatentDirichletAllocation.Document], + numTopics: Int, + topicSmoothing: Double, + wordSmoothing: Double, + randomSeed: Long): LearningState = { + val edges:RDD[Edge[WordCount]] = for { + d <- docs + (word, count) <- d.counts.toBreeze.activeIterator + if count != 0.0 + } yield { + Edge(d.id, word, count) + } + + val numWords = docs.take(1).head.counts.size + + val initialDocTopics = docs.map { doc => + val random: Random = new Random(doc.id + randomSeed) + (numWords + doc.id) -> BDV.fill(numTopics)(random.nextDouble()) + } + val initialWordCounts = docs.sparkContext.parallelize(0 until numWords).map { wid => + val random: Random = new Random(randomSeed + wid) + wid.toLong -> BDV.fill(numTopics)(random.nextDouble()) + } + + // partition such that edges are grouped by document + val graph = ( + Graph(initialDocTopics ++ initialWordCounts, edges) + .partitionBy(PartitionStrategy.EdgePartition1D) + ) + + LearningState(graph, numTopics, numWords, topicSmoothing, wordSmoothing) + + + } + + private def computePTopic(edge: EdgeTriplet[TopicCounts, WordCount]):TopicCounts = { + // \propto p(w|z) * p(z|d) + val scores = (edge.srcAttr :* edge.dstAttr) + // normalize and scale by number of times word occurs + scores *= (edge.attr / brzSum(scores)) + } +} + From 2891e890180aa6da843d08ee1b8aed71d095edc3 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 12 Jan 2015 12:34:32 -0800 Subject: [PATCH 02/30] Prepped LDA main class for PR, but some cleanups remain --- .../spark/examples/mllib/LDAExample.scala | 202 +++++ .../SimpleLatentDirichletAllocation.scala | 692 ------------------ .../apache/spark/mllib/clustering/LDA.scala | 566 ++++++++++++++ .../LatentDirichletAllocation.scala | 223 ------ 4 files changed, 768 insertions(+), 915 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala new file mode 100644 index 0000000000000..b028fe4fc943d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -0,0 +1,202 @@ +/* + * 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.examples.mllib + +import scala.collection.mutable.ArrayBuffer + +import java.text.BreakIterator + +import scopt.OptionParser + +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.clustering.LDA +import org.apache.spark.mllib.clustering.LDA.Document +import org.apache.spark.mllib.linalg.SparseVector +import org.apache.spark.rdd.RDD + + +/** + * An example Latent Dirichlet Allocation (LDA) app. Run with + * {{{ + * ./bin/run-example mllib.DenseKMeans [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object LDAExample { + + case class Params( + input: Seq[String] = Seq.empty, + k: Int = 20, + topicSmoothing: Double = 0.1, + termSmoothing: Double = 0.1, + vocabSize: Int = 10000) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("LDAExample") { + head("LDAExample: an example LDA app for plain text data.") + opt[Int]("k") + .text(s"number of topics. default: ${defaultParams.k}") + .action((x, c) => c.copy(k = x)) + opt[Double]("topicSmoothing") + .text(s"amount of topic smoothing to use. default: ${defaultParams.topicSmoothing}") + .action((x, c) => c.copy(topicSmoothing = x)) + opt[Double]("termSmoothing") + .text(s"amount of word smoothing to use. default: ${defaultParams.termSmoothing}") + .action((x, c) => c.copy(termSmoothing = x)) + opt[Int]("vocabSize") + .text(s"number of distinct word types to use, chosen by frequency." + + s" default: ${defaultParams.vocabSize}") + .action((x, c) => c.copy(vocabSize = x)) + arg[String]("...") + .text("input paths (directories) to plain text corpora") + .unbounded() + .required() + .action((x, c) => c.copy(input = c.input :+ x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + parser.showUsageAsError + sys.exit(1) + } + } + + private def run(params: Params) { + val conf = new SparkConf().setAppName(s"LDAExample with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + val corpus = preprocess(sc, params.input, params.vocabSize) + corpus.cache() + + val lda = new LDA() + lda.setK(params.k) + .setMaxIterations(4) + .setTopicSmoothing(params.topicSmoothing) + .setTermSmoothing(params.termSmoothing) + val ldaModel = lda.run(corpus) + + // TODO: print log likelihood + + } + + /** + * Load documents, tokenize them, create vocabulary, and prepare documents as word count vectors. + */ + private def preprocess( + sc: SparkContext, + paths: Seq[String], + vocabSize: Int): RDD[Document] = { + + val files: Seq[RDD[(String, String)]] = for (p <- paths) yield { + sc.wholeTextFiles(p) + } + + // Dataset of document texts + val textRDD: RDD[String] = + files.reduce(_ ++ _) // combine results from multiple paths + .map { case (path, text) => text } + + // Split text into words + val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => + id -> SimpleTokenizer.getWords(text) + } + + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Int)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1) } + .reduceByKey(_ + _) + + // Choose vocabulary: Map[word -> id] + val vocab: Map[String, Int] = wordCounts + .sortBy(_._2, ascending = false) + .take(vocabSize) + .map(_._1) + .zipWithIndex + .toMap + + val documents = tokenized.map { case (id, tokens) => + // Filter tokens by vocabulary, and create word count vector representation of document. + val wc = new scala.collection.mutable.HashMap[Int, Int]() + tokens.foreach { term => + if (vocab.contains(term)) { + val termIndex = vocab(term) + wc(termIndex) = wc.getOrElse(termIndex, 0) + 1 + } + } + val indices = wc.keys.toArray.sorted + val values = indices.map(i => wc(i).toDouble) + + val sb = new SparseVector(vocab.size, indices, values) + LDA.Document(sb, id) + } + + documents + } +} + +/** + * Simple Tokenizer. + * + * TODO: Formalize the interface, and make it a public class in mllib.feature + */ +private object SimpleTokenizer { + + // Matches sequences of Unicode letters + private val allWordRegex = "^(\\p{L}*)$".r + + // Ignore words shorter than this length. + private val minWordLength = 3 + + def getWords(text: String): IndexedSeq[String] = { + + val words = new ArrayBuffer[String]() + + // Use Java BreakIterator to tokenize text into words. + val wb = BreakIterator.getWordInstance + wb.setText(text) + + // current,end index start,end of each word + var current = wb.first() + var end = wb.next() + while (end != BreakIterator.DONE) { + // Convert to lowercase + val word: String = text.substring(current, end).toLowerCase + // Remove short words and strings that aren't only letters + word match { + case allWordRegex(w) if w.length >= minWordLength => + words += word + case _ => + } + + current = end + end = wb.next() + } + words + } + + // TODO: stopwords + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala deleted file mode 100644 index d3c30b292cb23..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SimpleLatentDirichletAllocation.scala +++ /dev/null @@ -1,692 +0,0 @@ -/* - * 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.examples.mllib - -import scopt.OptionParser -import org.apache.spark.{SparkContext, SparkConf} -import scala.collection.mutable.ArrayBuffer -import java.text.BreakIterator -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.topicmodel.LatentDirichletAllocation -import org.apache.spark.mllib.topicmodel.LatentDirichletAllocation.Document -import org.apache.spark.mllib.linalg.SparseVector -import org.apache.log4j.{Level, Logger} - -/** - * - * - * @author dlwh - */ -object SimpleLatentDirichletAllocation { - case class Params( - input: Seq[String] = Seq.empty, - numTopics: Int = 20, - wordSmoothing: Double = 0.1, - topicSmoothing: Double = 0.1, - vocabSize: Int = 10000, - minWordCount: Int = 10) extends AbstractParams[Params] - - def main(args: Array[String]) { - - val parser = new OptionParser[Params]("SimpleLatentDirichletAllocation") { - head("SimpleLatentDirichletAllocation: an example LDA app for plain text data.") - opt[Int]("numTopics") - .text("number of topics") - .action((x, c) => c.copy(numTopics = x)) - opt[Double]("wordSmoothing") - .text("amount of word smoothing to use") - .action((x, c) => c.copy(wordSmoothing = x)) - opt[Double]("topicSmoothing") - .text(s"amount of topic smoothing to use") - .action((x, c) => c.copy(topicSmoothing = x)) - opt[Int]("vocabSize") - .text(s"number of distinct word types to use, chosen by frequency (after stopword removal)") - .action((x, c) => c.copy(vocabSize = x)) - opt[Int]("minWordCount") - .text(s"minimum number of times a word must appear to be included in vocab.") - .action((x, c) => c.copy(minWordCount = x)) - arg[String]("...") - .text("input paths (directories) to plain text corpora") - .unbounded() - .required() - .action((x, c) => c.copy(input = c.input :+ x)) - } - - val params = parser.parse(args, Params()).getOrElse{parser.showUsageAsError; sys.exit(1)} - - val conf = new SparkConf().setAppName(s"LDA with $params") - val sc = new SparkContext(conf) - - Logger.getRootLogger.setLevel(Level.WARN) - - val corpus = preprocess(sc, params.input, params.vocabSize, params.minWordCount) - corpus.cache() - - val lda = new LatentDirichletAllocation(params.numTopics, - 100, - params.topicSmoothing, - params.wordSmoothing, - 0) - - for (state <- lda.iterations(corpus)) { - println(state.logLikelihood) - } - } - - def preprocess(sc: SparkContext, - paths: Seq[String], - vocabSize: Int, - minWordCount: Int): RDD[Document] = { - val files = for(p <- paths) yield { - sc.wholeTextFiles(p) - } - - val textRDD = files.reduce( _ ++ _ ) - - val tokenized = textRDD.zipWithIndex.map { case ((name, content), id) => - id -> SimpleTokenizer.getWords(content) - } - - val wordCounts: RDD[(String, Int)] = { - tokenized - .flatMap{ case (_, tokens) => tokens.map(_ -> 1)} - .reduceByKey(_ + _) - .filter(_._2 >= minWordCount) - } - - // word -> id - val vocab = ( - wordCounts - .sortBy(_._2, ascending = false) - .take(vocabSize) - .map(_._1) - .zipWithIndex - .toMap - ) - - val documents = tokenized.map { case (id, toks) => - val counts = breeze.linalg.Counter.countTraversable(toks) - - val indexedCounts = counts.iterator.collect { case (k, v) if vocab.contains(k) => - vocab(k) -> v.toDouble - } - - val sb = org.apache.spark.mllib.linalg.Vectors.sparse(vocab.size, indexedCounts.toSeq) - // I do not know why .sparse doesn't return a SparseVector. - LatentDirichletAllocation.Document(sb.asInstanceOf[SparseVector], id) - } - - documents - } - -} - -object SimpleTokenizer { - - val allWordRegex = "^(\\p{L}|\\p{M})*$".r - - def getWords(text: String): IndexedSeq[String] = { - val words = new ArrayBuffer[String]() - val wb = BreakIterator.getWordInstance - wb.setText(text) - - var current = wb.first() - var end = wb.next() - while (end != BreakIterator.DONE) { - val word: String = text.substring(current, end).toLowerCase - - // remove short words, things that aren't only letters, and stop words - if (allWordRegex.unapplySeq(word).nonEmpty && !stopWords(word) && word.length >= 3) { - words += word - } - - current = end - end = wb.next() - } - words - } - - val stopWords = - """ - |a - |able - |about - |above - |abst - |accordance - |according - |accordingly - |across - |act - |actually - |added - |adj - |affected - |affecting - |affects - |after - |afterwards - |again - |against - |ah - |all - |almost - |alone - |along - |already - |also - |although - |always - |am - |among - |amongst - |an - |and - |announce - |another - |any - |anybody - |anyhow - |anymore - |anyone - |anything - |anyway - |anyways - |anywhere - |apparently - |approximately - |are - |aren - |arent - |arise - |around - |as - |aside - |ask - |asking - |at - |auth - |available - |away - |awfully - |b - |back - |be - |became - |because - |become - |becomes - |becoming - |been - |before - |beforehand - |begin - |beginning - |beginnings - |begins - |behind - |being - |believe - |below - |beside - |besides - |between - |beyond - |biol - |both - |brief - |briefly - |but - |by - |c - |ca - |came - |can - |cannot - |can't - |cause - |causes - |certain - |certainly - |co - |com - |come - |comes - |contain - |containing - |contains - |could - |couldnt - |d - |date - |did - |didn't - |different - |do - |does - |doesn't - |doing - |done - |don't - |down - |downwards - |due - |during - |e - |each - |ed - |edu - |effect - |eg - |eight - |eighty - |either - |else - |elsewhere - |end - |ending - |enough - |especially - |et - |et-al - |etc - |even - |ever - |every - |everybody - |everyone - |everything - |everywhere - |ex - |except - |f - |far - |few - |ff - |fifth - |first - |five - |fix - |followed - |following - |follows - |for - |former - |formerly - |forth - |found - |four - |from - |further - |furthermore - |g - |gave - |get - |gets - |getting - |give - |given - |gives - |giving - |go - |goes - |gone - |got - |gotten - |h - |had - |happens - |hardly - |has - |hasn't - |have - |haven't - |having - |he - |hed - |hence - |her - |here - |hereafter - |hereby - |herein - |heres - |hereupon - |hers - |herself - |hes - |hi - |hid - |him - |himself - |his - |hither - |home - |how - |howbeit - |however - |hundred - |i - |id - |ie - |if - |i'll - |im - |immediate - |immediately - |importance - |important - |in - |inc - |indeed - |index - |information - |instead - |into - |invention - |inward - |is - |isn't - |it - |itd - |it'll - |its - |itself - |i've - |j - |just - |k - |keep keeps - |kept - |kg - |km - |know - |known - |knows - |l - |largely - |last - |lately - |later - |latter - |latterly - |least - |less - |lest - |let - |lets - |like - |liked - |likely - |line - |little - |'ll - |look - |looking - |looks - |ltd - |m - |made - |mainly - |make - |makes - |many - |may - |maybe - |me - |mean - |means - |meantime - |meanwhile - |merely - |mg - |might - |million - |miss - |ml - |more - |moreover - |most - |mostly - |mr - |mrs - |much - |mug - |must - |my - |myself - |n - |na - |name - |namely - |nay - |nd - |near - |nearly - |necessarily - |necessary - |need - |needs - |neither - |never - |nevertheless - |new - |next - |nine - |ninety - |no - |nobody - |non - |none - |nonetheless - |noone - |nor - |normally - |nos - |not - |noted - |nothing - |now - |nowhere - |o - |obtain - |obtained - |obviously - |of - |off - |often - |oh - |ok - |okay - |old - |omitted - |on - |once - |one - |ones - |only - |onto - |or - |ord - |other - |others - |otherwise - |ought - |our - |ours - |ourselves - |out - |outside - |over - |overall - |owing - |own - |p - |page - |pages - |part - |particular - |particularly - |past - |per - |perhaps - |placed - |please - |plus - |poorly - |possible - |possibly - |potentially - |pp - |predominantly - |present - |previously - |primarily - |probably - |promptly - |proud - |provides - |put - |q - |que - |quickly - |quite - |qv - |r - |ran - |rather - |rd - |re - |readily - |really - |recent - |recently - |ref - |refs - |regarding - |regardless - |regards - |related - |relatively - |research - |respectively - |resulted - |resulting - |results - |right - |run - |s - |said - |same - |saw - |say - |saying - |says - |sec - |section - |see - |seeing - |seem - |seemed - |seeming - |seems - |seen - |self - |selves - |sent - |seven - |several - |shall - |she - |shed - |she'll - |shes - |should - |shouldn't - |show - |showed - |shown - |showns - |shows - |significant - |significantly - |similar - |similarly - |since - |six - |slightly - |so - |some - |somebody - |somehow - |someone - |somethan - |something - |sometime - |sometimes - |somewhat - |somewhere - |soon - |sorry - |specifically - |specified - |specify - |specifying - |still - |stop - |strongly - |sub - |substantially - |successfully - |such - |sufficiently - |suggest - |sup - |sure - |t's take taken tell tends - |th than thank thanks thanx - |that that's thats the their - |theirs them themselves then thence - |there there's thereafter thereby therefore - |therein theres thereupon these they - |they'd they'll they're they've think - |third this thorough thoroughly those - |though three through throughout thru - |thus to together too took - |toward towards tried tries truly - |try trying twice two un - |under unfortunately unless unlikely until - |unto up upon us use - |used useful uses using usually - |value various very via viz - |vs want wants was wasn't - |way we we'd we'll we're - |we've welcome well went were - |weren't what what's whatever when - |whence whenever where where's whereafter - |whereas whereby wherein whereupon wherever - |whether which while whither who - |who's whoever whole whom whose - |why will willing wish with - |within without won't wonder would - |wouldn't yes yet you you'd - |you'll you're you've your yours - |yourself yourselves zero - """.stripMargin.split("\\s+").toSet - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala new file mode 100644 index 0000000000000..433d102206396 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -0,0 +1,566 @@ +/* + * 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.mllib.clustering + +import java.util.Random + +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, sum => brzSum, normalize} + +import org.apache.spark.graphx._ +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors, Matrix, Matrices} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.{BoundedPriorityQueue, Utils} + + +/** + * 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 + * + * Currently, the underlying implementation uses Expectation-Maximization (EM), implemented + * according to the Asuncion et al. (2009) paper referenced below. + * + * References: + * - Original LDA paper (journal version): + * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. + * - Paper which clearly explains several algorithms, including EM: + * Asuncion, Welling, Smyth, and Teh. + * "On Smoothing and Inference for Topic Models." UAI, 2009. + */ +class LDA private ( + private var k: Int, + private var maxIterations: Int, + private var topicSmoothing: Double, + private var termSmoothing: Double, + private var seed: Long) { + + import LDA._ + + def this() = this(k = 10, maxIterations = 10, topicSmoothing = -1, termSmoothing = 0.1, + seed = Utils.random.nextLong()) + + /** + * Number of topics to infer. I.e., the number of soft cluster centers. + * (default = 10) + */ + def getK: Int = k + + def setK(k: Int): this.type = { + this.k = k + this + } + + // TODO: UDPATE alpha, eta to be > 1 automatically for MAP + + /** + * Topic smoothing parameter (commonly named "alpha"). + * + * This is the parameter to the Dirichlet prior placed on the per-document topic distributions + * ("theta"). We use a symmetric Dirichlet prior. + * + * This value should be > 0.0, where larger values mean more smoothing (more regularization). + * If set <= 0, then topicSmoothing is set to equal 50 / k (where k is the number of topics). + * (default = 50 / k) + */ + def getTopicSmoothing: Double = topicSmoothing + + def setTopicSmoothing(alpha: Double): this.type = { + topicSmoothing = alpha + this + } + + /** + * Term smoothing parameter (commonly named "eta"). + * + * This is the parameter to the Dirichlet prior placed on the per-topic word distributions + * (which 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.) + * + * This value should be > 0.0. + * (default = 0.1) + */ + def getTermSmoothing: Double = termSmoothing + + def setTermSmoothing(eta: Double): this.type = { + termSmoothing = eta + this + } + + /** + * Maximum number of iterations for learning. + * (default = 10) + */ + def getMaxIterations: Int = maxIterations + + def setMaxIterations(maxIterations: Int): this.type = { + this.maxIterations = maxIterations + this + } + + /** Random seed */ + def getSeed: Long = seed + + def setSeed(seed: Long): this.type = { + this.seed = seed + this + } + + /** + * Learn an LDA model using the given dataset. + * + * @param docs RDD of documents, where each document is represented as a vector of term counts. + * Document IDs must be >= 0. + * @return Inferred LDA model + */ + def run(docs: RDD[Document]): DistributedLDAModel = { + var state = + LDA.initialState(docs, k, termSmoothing, topicSmoothing, seed) + var iter = 0 + while (iter < maxIterations) { + state = state.next() + iter += 1 + } + new DistributedLDAModel(state) + } +} + +/** + * Latent Dirichlet Allocation (LDA) model + */ +abstract class LDAModel private[clustering] { + + import LDA._ + + /** Number of topics */ + def k: Int + + /** Vocabulary size (number of terms or terms in the vocabulary) */ + def vocabSize: Int + + /** + * 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. + */ + def topicsMatrix: Matrix + + /* TODO + * Computes the estimated log likelihood of data (a set of documents), given the model. + * + * Note that this is an estimate since it requires inference (and exact inference is intractable + * for the LDA model). + * + * @param documents A set of documents, where each is represented as a vector of term counts. + * This must use the same vocabulary (ordering of term counts) as in training. + * Document IDs must be >= 0. + * @return Estimated log likelihood of the data under this model + */ + // TODO + //def logLikelihood(documents: RDD[Document]): Double + + /* TODO + * Compute the estimated topic distribution for each document. + * This is often called “theta” in the literature. + * + * @param documents A set of documents, where each is represented as a vector of term counts. + * This must use the same vocabulary (ordering of term counts) as in training. + * Document IDs must be >= 0. + * @return Estimated topic distribution for each document. + * The returned RDD may be zipped with the given RDD, where each returned vector + * is a multinomial distribution over topics. + */ + // def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] +} + +/** + * Local LDA model. + * This model stores only the inferred topics. + * It may be used for computing topics for new documents, but it may give less accurate answers + * than the [[DistributedLDAModel]]. + * + * @param topics Inferred topics (vocabSize x k matrix). + */ +class LocalLDAModel private[clustering] ( + private val topics: Matrix) extends LDAModel with Serializable { + + import LDA._ + + override def k: Int = topics.numCols + + override def vocabSize: Int = topics.numRows + + override def topicsMatrix: Matrix = topics + + // TODO + //override def logLikelihood(documents: RDD[Document]): Double = ??? + + // TODO: + // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? + +} + +/** + * Distributed LDA model. + * This model stores the inferred topics, the full training dataset, and the topic distributions. + * When computing topics for new documents, it may give more accurate answers + * than the [[LocalLDAModel]]. + */ +class DistributedLDAModel private[clustering] ( + private val state: LDA.LearningState) extends LDAModel { + + import LDA._ + + override def k: Int = state.k + + override def vocabSize: Int = state.vocabSize + + /** + * 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: This matrix is collected from an RDD. Beware memory usage when vocabSize, k are large. + */ + override lazy val topicsMatrix: Matrix = { + // Collect row-major topics + val termTopicCounts: Array[(Int, TopicCounts)] = + state.graph.vertices.filter(_._1 < 0).map { case (termIndex, cnts) => + (index2term(termIndex), cnts) + }.collect() + // Convert to Matrix + val brzTopics = BDM.zeros[Double](vocabSize, k) + termTopicCounts.foreach { case (term, cnts) => + var j = 0 + while (j < k) { + brzTopics(term, j) = cnts(j) + j += 1 + } + } + Matrices.fromBreeze(brzTopics) + } + + // TODO + //override def logLikelihood(documents: RDD[Document]): Double = ??? + + /** + * For each document in the training set, return the distribution over topics for that document + * (i.e., "theta_doc"). + * + * @return RDD of (document ID, topic distribution) pairs + */ + def topicDistributions: RDD[(Long, Vector)] = { + state.graph.vertices.filter(_._1 >= 0).map { case (docID, topicCounts) => + (docID.toLong, Vectors.fromBreeze(topicCounts)) + } + } + + // TODO: + // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? + + /* + // TODO: Do this properly + lazy val logLikelihood = { + graph.triplets.aggregate(0.0)({ (acc, triple) => + val scores = triple.srcAttr :* triple.dstAttr + val logScores = breeze.numerics.log(scores) + scores /= brzSum(scores) + brzSum(scores :*= logScores) * triple.attr + }, _ + _) + } + */ + + /** + * + * @param maxTermsPerTopic + * @return Array over topics, where each element is a set of top terms represented + * as (term weight in topic, term index). + */ + def getTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { + val nt = maxTermsPerTopic + state.graph.vertices.filter(_._1 < 0) // select term vertices + .mapPartitions { items => + // Create queue of + val queues = Array.fill(nt)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) + for ((termId, factor) <- items) { + var t = 0 + while (t < nt) { + queues(t) += (factor(t) -> termId.toInt) + t += 1 + } + } + Iterator(queues) + }.reduce { (q1, q2) => + q1.zip(q2).foreach { case (a,b) => a ++= b } + q1 + }.map ( q => q.toArray ) + } + +} + +object LDA { + + /* + DEVELOPERS NOTE: + + This implementation uses GraphX, where the graph is bipartite with 2 types of vertices: + - Document vertices + - indexed {0, 1, ..., numDocuments-1} + - Store vectors of length k (# topics). + - Term vertices + - indexed {-1, -2, ..., -vocabSize} + - Store vectors of length k (# topics). + - Edges correspond to terms appearing in documents. + - Edges are directed Document -> Term. + - Edges are partitioned by documents. + + Info on EM implementation. + - We follow Section 2.2 from Asuncion et al., 2009. + - In this implementation, there is one edge for every unique term appearing in a document, + i.e., for every unique (document, term) pair. + - Notation: + - N_{wkj} = count of tokens of term w currently assigned to topic k in document j + - N_{*} where * is missing a subscript w/k/j is the count summed over missing subscript(s) + - gamma_{wjk} = P(z_i = k | x_i = w, d_i = j), + the probability of term x_i in document d_i having topic z_i. + - Data graph + - Document vertices store N_{kj} + - Term vertices store N_{wk} + - Edges store N_{wj}. + - Global data N_k + - Algorithm + - Initial state: + - Document and term vertices store random counts N_{wk}, N_{kj}. + - E-step: For each (document,term) pair i, compute P(z_i | x_i, d_i). + - Aggregate N_k from term vertices. + - Compute gamma_{wjk} for each possible topic k, from each triplet. + using inputs N_{wk}, N_{kj}, N_k. + - M-step: Compute sufficient statistics for hidden parameters phi and theta + (counts N_{wk}, N_{kj}, N_k). + - Document update: + - N_{kj} <- sum_w N_{wj} gamma_{wjk} + - N_j <- sum_k N_{kj} (only needed to output predictions) + - Term update: + - N_{wk} <- sum_j N_{wj} gamma_{wjk} + - N_k <- sum_w N_{wk} + */ + + /** + * Document + * + * @param counts Vector of term (word) counts in the document. + * This is the "bag of words" representation. + * @param id Unique ID associated with this document. + * Documents should be indexed {0, 1, ..., numDocuments-1}. + * + * TODO: Can we remove the id and still be able to zip predicted topics with the Documents? + */ + case class Document(counts: SparseVector, id: VertexId) + + /** + * Vector over topics (length k) of token counts. + * The meaning of these counts can vary, and it may or may not be normalized to be a distribution. + */ + private[clustering] type TopicCounts = BDV[Double] + + private[clustering] type TokenCount = Double + + /** Term vertex IDs are {-1, -2, ..., -vocabSize} */ + private[clustering] def term2index(term: Int): Long = -(1 + term.toLong) + + private[clustering] def index2term(termIndex: Long): Int = -(1 + termIndex).toInt + + private[clustering] def isTermVertex(v: Tuple2[VertexId, _]): Boolean = v._1 < 0 + + private[clustering] def isDocVertex(v: Tuple2[VertexId, _]): Boolean = v._1 >= 0 + + /** + * + * Has all the information needed to run collapsed Gibbs sampling. + * + * @param graph + * @param k + * @param vocabSize + * @param topicSmoothing + * @param termSmoothing + */ + private[clustering] case class LearningState( + graph: Graph[TopicCounts, TokenCount], + k: Int, + vocabSize: Int, + topicSmoothing: Double, + termSmoothing: Double) { + + // TODO: Checkpoint periodically + def next() = copy(graph = step(graph)) + + private def step(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { + val eta = termSmoothing + val W = vocabSize + val alpha = topicSmoothing + + // Collect N_k from term vertices. + val N_k = collectTopicTotals() + val sendMsg: EdgeContext[TopicCounts, TokenCount, TopicCounts] => Unit = (edgeContext) => { + // Compute N_{wj} gamma_{wjk} + val N_wj = edgeContext.attr + // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count N_{wj}. + val scaledTopicDistribution: TopicCounts = + computePTopic(edgeContext, N_k, W, eta, alpha) * N_wj + edgeContext.sendToDst(scaledTopicDistribution) + edgeContext.sendToSrc(scaledTopicDistribution) + } + // M-STEP: Aggregation computes new N_{kj}, N_{wk} counts. + val docTopicDistributions: VertexRDD[TopicCounts] = + graph.aggregateMessages[TopicCounts](sendMsg, _ + _) + // Update the vertex descriptors with the new counts. + graph.outerJoinVertices(docTopicDistributions){ (vid, oldDist, newDist) => newDist.get } + } + + /* + /** + * Update document topic distributions, i.e., theta_doc = p(z|doc) for each doc. + * @return Graph with updated document vertex descriptors + */ + private def updateDocs(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { + val alpha = topicSmoothing + // Compute smoothed topic distributions for each document (size: numDocuments x k). + val docTopicTotals = updateExpectedCounts(_.srcId) + val newTotals = docTopicTotals.mapValues(total => normalize(total += alpha, 1)) + println(s"E-STEP newTotals.take(1): ${newTotals.take(1)(0)._2}") + // Update document vertices with new topic distributions. + graph.outerJoinVertices(newTotals){ (vid, old, newOpt) => newOpt.getOrElse(old) } + } + + /** + * Update topics, i.e., beta_z = p(w|z) for each topic z. + * (Conceptually, these are the topics. However, they are stored transposed, where each + * term vertex stores the distribution value for each topic.) + * @return Graph with updated term vertex descriptors + */ + private def updateTerms(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { + // Compute new topics. + val termTotals = updateExpectedCounts(_.dstId) + // Collect the aggregate counts over terms (summing all topics). + val eta: Double = termSmoothing + val topicTotals = termTotals.map(_._2).fold(BDV.zeros[Double](k))(_ + _) + topicTotals += (eta * vocabSize) + println(s"M-STEP topicTotals: $topicTotals") + // Update term vertices with new topic weights. + graph.outerJoinVertices(termTotals)( (vid, old, newOpt) => + newOpt + .map { counts => (counts += eta) :/= topicTotals } // smooth individual counts; normalize + .getOrElse(old) + ) + } + + private def updateExpectedCounts(sendToWhere: (EdgeTriplet[_, _]) => VertexId): VertexRDD[TopicCounts] = { + // Collect N_k from term vertices. + val N_k = collectTopicTotals() + val eta = termSmoothing + val W = vocabSize + val alpha = topicSmoothing + graph.mapReduceTriplets[TopicCounts]({ + trip => Iterator(sendToWhere(trip) -> computePTopic(trip, N_k, W, eta, alpha)) + }, _ += _) + } + */ + + private def collectTopicTotals(): TopicCounts = { + val numTopics = k + graph.vertices.filter(isTermVertex).map(_._2).fold(BDV.zeros[Double](numTopics))(_ + _) + } + + } + + private def computePTopic(edgeContext: EdgeContext[TopicCounts, TokenCount, TopicCounts], + N_k: TopicCounts, vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { + val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) + val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + // proportional to p(w|z) * p(z|d) / p(z) + val unnormalizedGamma = smoothed_N_wk :* smoothed_N_kj :/ smoothed_N_k + // normalize + unnormalizedGamma /= brzSum(unnormalizedGamma) + } + + /* + private def computePTopic(edge: EdgeTriplet[TopicCounts, TokenCount], N_k: TopicCounts, vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { + val smoothed_N_wk: TopicCounts = edge.dstAttr + (eta - 1.0) + val smoothed_N_kj: TopicCounts = edge.srcAttr + (alpha - 1.0) + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + // proportional to p(w|z) * p(z|d) / p(z) + val unnormalizedGamma = smoothed_N_wk :* smoothed_N_kj :/ smoothed_N_k + // normalize + unnormalizedGamma /= brzSum(unnormalizedGamma) + } + */ + + /** + * Compute bipartite term/doc graph. + * doc ids are shifted by vocabSize to maintain uniqueness + */ + private def initialState( + docs: RDD[Document], + k: Int, + topicSmoothing: Double, + termSmoothing: Double, + randomSeed: Long): LearningState = { + // For each document, create an edge (Document -> Term) for each unique term in the document. + val edges: RDD[Edge[TokenCount]] = docs.mapPartitionsWithIndex { case (partIndex, partDocs) => + partDocs.flatMap { doc: Document => + // Add edges for terms with non-zero counts. + doc.counts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + Edge(doc.id, term2index(term), cnt) + } + } + } + + val vocabSize = docs.take(1).head.counts.size + + // Create vertices. + // Initially, we use random soft assignments of tokens to topics (random gamma). + val edgesWithGamma: RDD[(Edge[TokenCount], TopicCounts)] = + edges.mapPartitionsWithIndex { case (partIndex, partEdges) => + val random = new Random(partIndex + randomSeed) + partEdges.map { edge => + // Create a random gamma_{wjk} + (edge, normalize(BDV.fill[Double](k)(random.nextDouble()), 1.0)) + } + } + def createVertices(sendToWhere: Edge[TokenCount] => VertexId): RDD[(VertexId, TopicCounts)] = { + val verticesTMP: RDD[(VertexId, TopicCounts)] = + edgesWithGamma.map { case (edge, gamma: TopicCounts) => + val N_wj = edge.attr + (sendToWhere(edge), gamma * N_wj) + } + verticesTMP.foldByKey(BDV.zeros[Double](k))(_ + _) + } + val docVertices = createVertices(_.srcId) + val termVertices = createVertices(_.dstId) + + // Partition such that edges are grouped by document + val graph = Graph(docVertices ++ termVertices, edges) + .partitionBy(PartitionStrategy.EdgePartition1D) + + LearningState(graph, k, vocabSize, topicSmoothing, termSmoothing) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala b/mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala deleted file mode 100644 index 24bb398904494..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/topicmodel/LatentDirichletAllocation.scala +++ /dev/null @@ -1,223 +0,0 @@ -/* - * 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.mllib.topicmodel - -import org.apache.spark.rdd.RDD -import java.util.Random -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.util.BoundedPriorityQueue -import org.apache.hadoop.fs.shell.Count - - - - -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, sum => brzSum, normalize} -import org.apache.spark.mllib.linalg.{DenseVector => SDV, SparseVector => SSV, Vector => SV} - -import org.apache.spark.graphx._ - - -/** - * - * - */ -class LatentDirichletAllocation(var numTopics: Int, - var maxIterations: Int, - var topicSmoothing: Double, - var wordSmoothing: Double, - randomSeed: Long) { - def this(numTopics: Int, maxIterations: Int) = { - this(numTopics, maxIterations, 0.1, 0.1, System.currentTimeMillis()) - } - - def this(numTopics: Int) = this(numTopics, 100) - - - def setNumTopics(k: Int):this.type = {numTopics = k; this} - - def setTopicSmoothing(alpha: Double):this.type = {topicSmoothing = alpha; this} - - def setWordSmoothing(beta: Double):this.type = {wordSmoothing = beta; this} - - import LatentDirichletAllocation._ - - def iterations(docs: RDD[LatentDirichletAllocation.Document]):Iterator[State] = { - val state = initialState(docs, numTopics, wordSmoothing, topicSmoothing, randomSeed) - Iterator.iterate(state)(_.next()).drop(2).take(maxIterations) - } - - def run(docs: RDD[LatentDirichletAllocation.Document]):State = { - import breeze.util.Implicits._ - iterations(docs).last - } -} - -object LatentDirichletAllocation { - case class Document(counts: SSV, id: VertexId) - - private type TopicCounts = BDV[Double] - // Strictly should be an integer, but the algorithm works with Doubles - private type WordCount = Double - - trait State { - def logLikelihood: Double - - def topWords(k: Int):Array[Array[(Double, Int)]] - } - - /** - * - * Has all the information needed to run EM. - * - * The Graph has two kinds of nodes: words and documents. The attr for a word - * is p(w|z). The attr for a document is p(z|doc) - * - * @param graph - * @param numTopics - * @param numWords - * @param topicSmoothing - * @param wordSmoothing - * @param numEStepIters - */ - private case class LearningState(graph: Graph[TopicCounts, Double], - numTopics: Int, - numWords: Int, - topicSmoothing: Double, - wordSmoothing: Double, - numEStepIters: Int = 10) extends State { - - def next() = copy(graph = mStep(eStep(graph))) - - // update p(z|doc) for each doc - private def eStep(graph: Graph[TopicCounts, Double]): Graph[TopicCounts, Double] = { - (0 until numEStepIters).foldLeft(graph) { (graph, _) => - // TODO: we should be able to detect which documents have converged and - // filter them so we don't bother with them for the rest of the estep - val docTopicTotals = updateExpectedCounts(graph, _.srcId) - val alpha = topicSmoothing - val newTotals = docTopicTotals.mapValues(total => normalize(total += alpha, 1)) - graph.outerJoinVertices(newTotals){ (vid, old, newOpt) => newOpt.getOrElse(old)} - } - } - - // update p(w|z) for each word - private def mStep(graph: Graph[TopicCounts, Double]): Graph[TopicCounts, Double] = { - val wordTotals = updateExpectedCounts(graph, _.dstId) - val beta: Double = wordSmoothing - val topicTotals = wordTotals.map(_._2).fold(BDV.zeros[Double](numTopics))(_ + _) - // smooth the totals - topicTotals += (beta * numWords) - - graph.outerJoinVertices(wordTotals)( (vid, old, newOpt) => - newOpt - .map ( counts => (counts += beta) :/= topicTotals) // smooth individual counts; normalize - .getOrElse(old) // keep old p(z|doc) vectors - ) - } - - lazy val logLikelihood = { - graph.triplets.aggregate(0.0)({ (acc, triple) => - val scores = triple.srcAttr :* triple.dstAttr - val logScores = breeze.numerics.log(scores) - scores /= brzSum(scores) - brzSum(scores :*= logScores) * triple.attr - }, _ + _) - } - - // cribbed from jegonzal's implementation - def topWords(k: Int): Array[Array[(Double, Int)]] = { - val nt = numTopics - val nw = numWords - graph.vertices.filter { - case (vid, c) => vid < nw - }.mapPartitions { items => - val queues = Array.fill(nt)(new BoundedPriorityQueue[(Double, Int)](k)) - for ((wordId, factor) <- items) { - var t = 0 - while (t < nt) { - queues(t) += (factor(t) -> wordId.toInt) - t += 1 - } - } - Iterator(queues) - }.reduce { (q1, q2) => - q1.zip(q2).foreach { case (a,b) => a ++= b } - q1 - }.map ( q => q.toArray ) - } - - - } - - - private def updateExpectedCounts(wordCountGraph: Graph[TopicCounts, Double], - sendToWhere: (EdgeTriplet[_, _]) => VertexId) = { - wordCountGraph.mapReduceTriplets[TopicCounts]({ - trip => Iterator(sendToWhere(trip) -> computePTopic(trip)) - }, _ += _) - } - - /** - * Compute bipartite term/doc graph. doc ids are shifted by numWords to maintain uniqueness - * @param docs - * @param numTopics - * @param randomSeed - * @return - */ - private def initialState(docs: RDD[LatentDirichletAllocation.Document], - numTopics: Int, - topicSmoothing: Double, - wordSmoothing: Double, - randomSeed: Long): LearningState = { - val edges:RDD[Edge[WordCount]] = for { - d <- docs - (word, count) <- d.counts.toBreeze.activeIterator - if count != 0.0 - } yield { - Edge(d.id, word, count) - } - - val numWords = docs.take(1).head.counts.size - - val initialDocTopics = docs.map { doc => - val random: Random = new Random(doc.id + randomSeed) - (numWords + doc.id) -> BDV.fill(numTopics)(random.nextDouble()) - } - val initialWordCounts = docs.sparkContext.parallelize(0 until numWords).map { wid => - val random: Random = new Random(randomSeed + wid) - wid.toLong -> BDV.fill(numTopics)(random.nextDouble()) - } - - // partition such that edges are grouped by document - val graph = ( - Graph(initialDocTopics ++ initialWordCounts, edges) - .partitionBy(PartitionStrategy.EdgePartition1D) - ) - - LearningState(graph, numTopics, numWords, topicSmoothing, wordSmoothing) - - - } - - private def computePTopic(edge: EdgeTriplet[TopicCounts, WordCount]):TopicCounts = { - // \propto p(w|z) * p(z|d) - val scores = (edge.srcAttr :* edge.dstAttr) - // normalize and scale by number of times word occurs - scores *= (edge.attr / brzSum(scores)) - } -} - From 2d400066a13e965d969f0edad1637773051f9b53 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 13 Jan 2015 11:32:12 -0800 Subject: [PATCH 03/30] cleanups before PR --- .../spark/examples/mllib/LDAExample.scala | 68 ++++++-- .../apache/spark/mllib/clustering/LDA.scala | 147 +++++++----------- 2 files changed, 108 insertions(+), 107 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index b028fe4fc943d..5ee8c53dfeae2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -42,12 +42,14 @@ import org.apache.spark.rdd.RDD */ object LDAExample { - case class Params( - input: Seq[String] = Seq.empty, - k: Int = 20, - topicSmoothing: Double = 0.1, - termSmoothing: Double = 0.1, - vocabSize: Int = 10000) extends AbstractParams[Params] + private case class Params( + input: Seq[String] = Seq.empty, + k: Int = 20, + maxIterations: Int = 10, + topicSmoothing: Double = 0.1, + termSmoothing: Double = 0.1, + vocabSize: Int = 10000, + stopwordFile: String = "") extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() @@ -57,6 +59,9 @@ object LDAExample { opt[Int]("k") .text(s"number of topics. default: ${defaultParams.k}") .action((x, c) => c.copy(k = x)) + opt[Int]("maxIterations") + .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}") + .action((x, c) => c.copy(maxIterations = x)) opt[Double]("topicSmoothing") .text(s"amount of topic smoothing to use. default: ${defaultParams.topicSmoothing}") .action((x, c) => c.copy(topicSmoothing = x)) @@ -67,6 +72,10 @@ object LDAExample { .text(s"number of distinct word types to use, chosen by frequency." + s" default: ${defaultParams.vocabSize}") .action((x, c) => c.copy(vocabSize = x)) + opt[String]("stopwordFile") + .text(s"filepath for a list of stopwords. Note: This must fit on a single machine." + + s" default: ${defaultParams.stopwordFile}") + .action((x, c) => c.copy(stopwordFile = x)) arg[String]("...") .text("input paths (directories) to plain text corpora") .unbounded() @@ -88,16 +97,31 @@ object LDAExample { Logger.getRootLogger.setLevel(Level.WARN) - val corpus = preprocess(sc, params.input, params.vocabSize) - corpus.cache() + // Load documents, and prepare them for LDA. + val (corpus, vocabArray) = preprocess(sc, params.input, params.vocabSize, params.stopwordFile) + corpus.cache() // cache since LDA is iterative + // Run LDA. val lda = new LDA() lda.setK(params.k) - .setMaxIterations(4) + .setMaxIterations(params.maxIterations) .setTopicSmoothing(params.topicSmoothing) .setTermSmoothing(params.termSmoothing) val ldaModel = lda.run(corpus) + // Print the topics, showing the top-weighted terms for each topic. + val topicIndices = ldaModel.getTopics(maxTermsPerTopic = 10) + val topics = topicIndices.map { topic => + topic.map { case (weight, term) => (weight, vocabArray(term)) } + } + println(s"${params.k} topics:") + topics.zipWithIndex.foreach { case (topic, i) => + println(s"TOPIC $i") + topic.foreach { case (weight, term) => + println(s"$term\t$weight") + } + } + // TODO: print log likelihood } @@ -108,7 +132,8 @@ object LDAExample { private def preprocess( sc: SparkContext, paths: Seq[String], - vocabSize: Int): RDD[Document] = { + vocabSize: Int, + stopwordFile: String): (RDD[Document], Array[String]) = { val files: Seq[RDD[(String, String)]] = for (p <- paths) yield { sc.wholeTextFiles(p) @@ -120,8 +145,9 @@ object LDAExample { .map { case (path, text) => text } // Split text into words + val tokenizer = new SimpleTokenizer(sc, stopwordFile) val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => - id -> SimpleTokenizer.getWords(text) + id -> tokenizer.getWords(text) } // Counts words: RDD[(word, wordCount)] @@ -153,16 +179,26 @@ object LDAExample { LDA.Document(sb, id) } - documents + val vocabArray = new Array[String](vocab.size) + vocab.foreach { case (term, i) => vocabArray(i) = term } + + (documents, vocabArray) } } /** * Simple Tokenizer. * - * TODO: Formalize the interface, and make it a public class in mllib.feature + * TODO: Formalize the interface, and make this a public class in mllib.feature */ -private object SimpleTokenizer { +private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Serializable { + + private val stopwords: Set[String] = if (stopwordFile.isEmpty) { + Set.empty[String] + } else { + val stopwordText = sc.textFile(stopwordFile).collect() + stopwordText.flatMap(_.stripMargin.split("\\s+")).toSet + } // Matches sequences of Unicode letters private val allWordRegex = "^(\\p{L}*)$".r @@ -186,7 +222,7 @@ private object SimpleTokenizer { val word: String = text.substring(current, end).toLowerCase // Remove short words and strings that aren't only letters word match { - case allWordRegex(w) if w.length >= minWordLength => + case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => words += word case _ => } @@ -197,6 +233,4 @@ private object SimpleTokenizer { words } - // TODO: stopwords - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 433d102206396..9dda872cad5c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -22,7 +22,7 @@ import java.util.Random import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, sum => brzSum, normalize} import org.apache.spark.graphx._ -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors, Matrix, Matrices} +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors, Matrix, Matrices} import org.apache.spark.rdd.RDD import org.apache.spark.util.{BoundedPriorityQueue, Utils} @@ -79,6 +79,11 @@ class LDA private ( * This value should be > 0.0, where larger values mean more smoothing (more regularization). * If set <= 0, then topicSmoothing is set to equal 50 / k (where k is the number of topics). * (default = 50 / k) + * + * Details on this parameter: + * The above description tells how the base value of topicSmoothing is set. However, following + * Asuncion et al. (2009), we adjust the base value based on the algorithm. + * - For EM, we increase topicSmoothing by +1.0. */ def getTopicSmoothing: Double = topicSmoothing @@ -96,6 +101,11 @@ class LDA private ( * * This value should be > 0.0. * (default = 0.1) + * + * Details on this parameter: + * The above description tells how the base value of topicSmoothing is set. However, following + * Asuncion et al. (2009), we adjust the base value based on the algorithm. + * - For EM, we increase topicSmoothing by +1.0. */ def getTermSmoothing: Double = termSmoothing @@ -131,6 +141,12 @@ class LDA private ( * @return Inferred LDA model */ def run(docs: RDD[Document]): DistributedLDAModel = { + val termSmoothing = this.termSmoothing + 1.0 + val topicSmoothing = if (this.topicSmoothing > 0) { + this.topicSmoothing + } else { + 50.0 / k + } + 1.0 var state = LDA.initialState(docs, k, termSmoothing, topicSmoothing, seed) var iter = 0 @@ -288,29 +304,37 @@ class DistributedLDAModel private[clustering] ( */ /** + * Return the topics described by the top-weighted terms. * - * @param maxTermsPerTopic + * Note: This is approximate; it may not return exactly the top-weighted terms for each topic. + * To get a more precise set of top terms, increase maxTermsPerTopic. + * + * @param maxTermsPerTopic Maximum number of terms to collect for each topic. * @return Array over topics, where each element is a set of top terms represented * as (term weight in topic, term index). + * Each topic's terms are sorted in order of decreasing weight. */ def getTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { - val nt = maxTermsPerTopic - state.graph.vertices.filter(_._1 < 0) // select term vertices - .mapPartitions { items => - // Create queue of - val queues = Array.fill(nt)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) - for ((termId, factor) <- items) { - var t = 0 - while (t < nt) { - queues(t) += (factor(t) -> termId.toInt) - t += 1 + val numTopics = k + val N_k: TopicCounts = state.collectTopicTotals() + state.graph.vertices.filter(isTermVertex) + .mapPartitions { termVertices => + // For this partition, collect the most common terms for each topic in queues: + // queues(topic) = queue of (term weight, term index). + // Term weights are N_{wk} / N_k. + val queues = Array.fill(numTopics)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) + for ((termId, n_wk) <- termVertices) { + var topic = 0 + while (topic < numTopics) { + queues(topic) += (n_wk(topic) / N_k(topic) -> index2term(termId.toInt)) + topic += 1 } } Iterator(queues) }.reduce { (q1, q2) => - q1.zip(q2).foreach { case (a,b) => a ++= b } + q1.zip(q2).foreach { case (a, b) => a ++= b} q1 - }.map ( q => q.toArray ) + }.map(_.toArray.sortBy(_._1)) } } @@ -387,19 +411,17 @@ object LDA { private[clustering] def index2term(termIndex: Long): Int = -(1 + termIndex).toInt - private[clustering] def isTermVertex(v: Tuple2[VertexId, _]): Boolean = v._1 < 0 - - private[clustering] def isDocVertex(v: Tuple2[VertexId, _]): Boolean = v._1 >= 0 + private[clustering] def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 /** + * State for EM algorithm: data + parameter graph, plus algorithm parameters. * - * Has all the information needed to run collapsed Gibbs sampling. - * - * @param graph - * @param k - * @param vocabSize - * @param topicSmoothing - * @param termSmoothing + * @param graph EM graph, storing current parameter estimates in vertex descriptors and + * data (token counts) in edge descriptors. + * @param k Number of topics + * @param vocabSize Number of unique terms + * @param topicSmoothing "alpha" + * @param termSmoothing "eta" */ private[clustering] case class LearningState( graph: Graph[TopicCounts, TokenCount], @@ -408,7 +430,7 @@ object LDA { topicSmoothing: Double, termSmoothing: Double) { - // TODO: Checkpoint periodically + // TODO: Checkpoint periodically? def next() = copy(graph = step(graph)) private def step(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { @@ -434,64 +456,21 @@ object LDA { graph.outerJoinVertices(docTopicDistributions){ (vid, oldDist, newDist) => newDist.get } } - /* - /** - * Update document topic distributions, i.e., theta_doc = p(z|doc) for each doc. - * @return Graph with updated document vertex descriptors - */ - private def updateDocs(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { - val alpha = topicSmoothing - // Compute smoothed topic distributions for each document (size: numDocuments x k). - val docTopicTotals = updateExpectedCounts(_.srcId) - val newTotals = docTopicTotals.mapValues(total => normalize(total += alpha, 1)) - println(s"E-STEP newTotals.take(1): ${newTotals.take(1)(0)._2}") - // Update document vertices with new topic distributions. - graph.outerJoinVertices(newTotals){ (vid, old, newOpt) => newOpt.getOrElse(old) } - } - - /** - * Update topics, i.e., beta_z = p(w|z) for each topic z. - * (Conceptually, these are the topics. However, they are stored transposed, where each - * term vertex stores the distribution value for each topic.) - * @return Graph with updated term vertex descriptors - */ - private def updateTerms(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { - // Compute new topics. - val termTotals = updateExpectedCounts(_.dstId) - // Collect the aggregate counts over terms (summing all topics). - val eta: Double = termSmoothing - val topicTotals = termTotals.map(_._2).fold(BDV.zeros[Double](k))(_ + _) - topicTotals += (eta * vocabSize) - println(s"M-STEP topicTotals: $topicTotals") - // Update term vertices with new topic weights. - graph.outerJoinVertices(termTotals)( (vid, old, newOpt) => - newOpt - .map { counts => (counts += eta) :/= topicTotals } // smooth individual counts; normalize - .getOrElse(old) - ) - } - - private def updateExpectedCounts(sendToWhere: (EdgeTriplet[_, _]) => VertexId): VertexRDD[TopicCounts] = { - // Collect N_k from term vertices. - val N_k = collectTopicTotals() - val eta = termSmoothing - val W = vocabSize - val alpha = topicSmoothing - graph.mapReduceTriplets[TopicCounts]({ - trip => Iterator(sendToWhere(trip) -> computePTopic(trip, N_k, W, eta, alpha)) - }, _ += _) - } - */ - - private def collectTopicTotals(): TopicCounts = { + private[clustering] def collectTopicTotals(): TopicCounts = { val numTopics = k graph.vertices.filter(isTermVertex).map(_._2).fold(BDV.zeros[Double](numTopics))(_ + _) } - } - private def computePTopic(edgeContext: EdgeContext[TopicCounts, TokenCount, TopicCounts], - N_k: TopicCounts, vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { + /** + * Compute gamma_{wjk}, a distribution over topics k. + */ + private def computePTopic( + edgeContext: EdgeContext[TopicCounts, TokenCount, TopicCounts], + N_k: TopicCounts, + vocabSize: Int, + eta: Double, + alpha: Double): TopicCounts = { val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) @@ -501,18 +480,6 @@ object LDA { unnormalizedGamma /= brzSum(unnormalizedGamma) } - /* - private def computePTopic(edge: EdgeTriplet[TopicCounts, TokenCount], N_k: TopicCounts, vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { - val smoothed_N_wk: TopicCounts = edge.dstAttr + (eta - 1.0) - val smoothed_N_kj: TopicCounts = edge.srcAttr + (alpha - 1.0) - val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) - // proportional to p(w|z) * p(z|d) / p(z) - val unnormalizedGamma = smoothed_N_wk :* smoothed_N_kj :/ smoothed_N_k - // normalize - unnormalizedGamma /= brzSum(unnormalizedGamma) - } - */ - /** * Compute bipartite term/doc graph. * doc ids are shifted by vocabSize to maintain uniqueness From 377ebd9eb2584e155f3bbad6361e52ca3bb8a2c1 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 13 Jan 2015 16:45:06 -0800 Subject: [PATCH 04/30] separated LDA models into own file. more cleanups before PR --- .../spark/examples/mllib/LDAExample.scala | 16 +- .../apache/spark/mllib/clustering/LDA.scala | 312 ++++++------------ .../spark/mllib/clustering/LDAModel.scala | 265 +++++++++++++++ 3 files changed, 383 insertions(+), 210 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 5ee8c53dfeae2..ac0f48d0b45d1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -107,12 +107,21 @@ object LDAExample { .setMaxIterations(params.maxIterations) .setTopicSmoothing(params.topicSmoothing) .setTermSmoothing(params.termSmoothing) + val startTime = System.nanoTime() val ldaModel = lda.run(corpus) + val elapsed = (System.nanoTime() - startTime) / 1e9 + + println(s"Finished training LDA model. Summary:") + println(s"\t Training time: $elapsed sec") + println(s"\t Training set size: ${corpus.count()} documents") + println(s"\t Vocabulary size: ${vocabArray.size} terms") + println(s"\t Training data average log likelihood: ${ldaModel.logLikelihood / corpus.count()}") + println() // Print the topics, showing the top-weighted terms for each topic. - val topicIndices = ldaModel.getTopics(maxTermsPerTopic = 10) + val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) val topics = topicIndices.map { topic => - topic.map { case (weight, term) => (weight, vocabArray(term)) } + topic.map { case (weight, term) => (weight, vocabArray(term.toInt)) } } println(s"${params.k} topics:") topics.zipWithIndex.foreach { case (topic, i) => @@ -120,10 +129,9 @@ object LDAExample { topic.foreach { case (weight, term) => println(s"$term\t$weight") } + println() } - // TODO: print log likelihood - } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 9dda872cad5c7..b8dc6345fdc8d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -19,15 +19,18 @@ package org.apache.spark.mllib.clustering import java.util.Random -import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, sum => brzSum, normalize} +import breeze.linalg.{DenseVector => BDV, sum => brzSum, normalize} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.graphx._ -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors, Matrix, Matrices} +import org.apache.spark.mllib.linalg.SparseVector import org.apache.spark.rdd.RDD -import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.Utils /** + * :: DeveloperApi :: + * * Latent Dirichlet Allocation (LDA), a topic model designed for text documents. * * Terminology: @@ -40,11 +43,16 @@ import org.apache.spark.util.{BoundedPriorityQueue, Utils} * * References: * - Original LDA paper (journal version): - * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. + * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. + * - This class implements their "smoothed" LDA model. * - Paper which clearly explains several algorithms, including EM: - * Asuncion, Welling, Smyth, and Teh. - * "On Smoothing and Inference for Topic Models." UAI, 2009. + * Asuncion, Welling, Smyth, and Teh. + * "On Smoothing and Inference for Topic Models." UAI, 2009. + * + * NOTE: This is currently marked DeveloperApi since it is under active development and may undergo + * API changes. */ +@DeveloperApi class LDA private ( private var k: Int, private var maxIterations: Int, @@ -64,12 +72,11 @@ class LDA private ( def getK: Int = k def setK(k: Int): this.type = { + require(k > 0, s"LDA k (number of clusters) must be > 0, but was set to $k") this.k = k this } - // TODO: UDPATE alpha, eta to be > 1 automatically for MAP - /** * Topic smoothing parameter (commonly named "alpha"). * @@ -77,7 +84,7 @@ class LDA private ( * ("theta"). We use a symmetric Dirichlet prior. * * This value should be > 0.0, where larger values mean more smoothing (more regularization). - * If set <= 0, then topicSmoothing is set to equal 50 / k (where k is the number of topics). + * If set to -1, then topicSmoothing is set to equal 50 / k (where k is the number of topics). * (default = 50 / k) * * Details on this parameter: @@ -87,8 +94,10 @@ class LDA private ( */ def getTopicSmoothing: Double = topicSmoothing - def setTopicSmoothing(alpha: Double): this.type = { - topicSmoothing = alpha + def setTopicSmoothing(topicSmoothing: Double): this.type = { + require(topicSmoothing > 0.0 || topicSmoothing == -1.0, + s"LDA topicSmoothing must be > 0, but was set to $topicSmoothing") + this.topicSmoothing = topicSmoothing this } @@ -109,8 +118,9 @@ class LDA private ( */ def getTermSmoothing: Double = termSmoothing - def setTermSmoothing(eta: Double): this.type = { - termSmoothing = eta + def setTermSmoothing(termSmoothing: Double): this.type = { + require(termSmoothing > 0.0, s"LDA termSmoothing must be > 0, but was set to $termSmoothing") + this.termSmoothing = termSmoothing this } @@ -136,208 +146,29 @@ class LDA private ( /** * Learn an LDA model using the given dataset. * - * @param docs RDD of documents, where each document is represented as a vector of term counts. - * Document IDs must be >= 0. + * @param documents RDD of documents, where each document is represented as a vector of term + * counts plus an ID. Document IDs must be >= 0. * @return Inferred LDA model */ - def run(docs: RDD[Document]): DistributedLDAModel = { + def run(documents: RDD[Document]): DistributedLDAModel = { val termSmoothing = this.termSmoothing + 1.0 - val topicSmoothing = if (this.topicSmoothing > 0) { + val topicSmoothing = (if (this.topicSmoothing > 0) { this.topicSmoothing } else { 50.0 / k - } + 1.0 - var state = - LDA.initialState(docs, k, termSmoothing, topicSmoothing, seed) + }) + 1.0 + var state = LDA.initialState(documents, k, topicSmoothing, termSmoothing, seed) + println(s"DEBUG: initial log likelihood = ${state.logLikelihood}") var iter = 0 while (iter < maxIterations) { state = state.next() + println(s"DEBUG: iter=$iter. log likelihood = ${state.logLikelihood}") iter += 1 } new DistributedLDAModel(state) } } -/** - * Latent Dirichlet Allocation (LDA) model - */ -abstract class LDAModel private[clustering] { - - import LDA._ - - /** Number of topics */ - def k: Int - - /** Vocabulary size (number of terms or terms in the vocabulary) */ - def vocabSize: Int - - /** - * 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. - */ - def topicsMatrix: Matrix - - /* TODO - * Computes the estimated log likelihood of data (a set of documents), given the model. - * - * Note that this is an estimate since it requires inference (and exact inference is intractable - * for the LDA model). - * - * @param documents A set of documents, where each is represented as a vector of term counts. - * This must use the same vocabulary (ordering of term counts) as in training. - * Document IDs must be >= 0. - * @return Estimated log likelihood of the data under this model - */ - // TODO - //def logLikelihood(documents: RDD[Document]): Double - - /* TODO - * Compute the estimated topic distribution for each document. - * This is often called “theta” in the literature. - * - * @param documents A set of documents, where each is represented as a vector of term counts. - * This must use the same vocabulary (ordering of term counts) as in training. - * Document IDs must be >= 0. - * @return Estimated topic distribution for each document. - * The returned RDD may be zipped with the given RDD, where each returned vector - * is a multinomial distribution over topics. - */ - // def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] -} - -/** - * Local LDA model. - * This model stores only the inferred topics. - * It may be used for computing topics for new documents, but it may give less accurate answers - * than the [[DistributedLDAModel]]. - * - * @param topics Inferred topics (vocabSize x k matrix). - */ -class LocalLDAModel private[clustering] ( - private val topics: Matrix) extends LDAModel with Serializable { - - import LDA._ - - override def k: Int = topics.numCols - - override def vocabSize: Int = topics.numRows - - override def topicsMatrix: Matrix = topics - - // TODO - //override def logLikelihood(documents: RDD[Document]): Double = ??? - - // TODO: - // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? - -} - -/** - * Distributed LDA model. - * This model stores the inferred topics, the full training dataset, and the topic distributions. - * When computing topics for new documents, it may give more accurate answers - * than the [[LocalLDAModel]]. - */ -class DistributedLDAModel private[clustering] ( - private val state: LDA.LearningState) extends LDAModel { - - import LDA._ - - override def k: Int = state.k - - override def vocabSize: Int = state.vocabSize - - /** - * 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: This matrix is collected from an RDD. Beware memory usage when vocabSize, k are large. - */ - override lazy val topicsMatrix: Matrix = { - // Collect row-major topics - val termTopicCounts: Array[(Int, TopicCounts)] = - state.graph.vertices.filter(_._1 < 0).map { case (termIndex, cnts) => - (index2term(termIndex), cnts) - }.collect() - // Convert to Matrix - val brzTopics = BDM.zeros[Double](vocabSize, k) - termTopicCounts.foreach { case (term, cnts) => - var j = 0 - while (j < k) { - brzTopics(term, j) = cnts(j) - j += 1 - } - } - Matrices.fromBreeze(brzTopics) - } - - // TODO - //override def logLikelihood(documents: RDD[Document]): Double = ??? - - /** - * For each document in the training set, return the distribution over topics for that document - * (i.e., "theta_doc"). - * - * @return RDD of (document ID, topic distribution) pairs - */ - def topicDistributions: RDD[(Long, Vector)] = { - state.graph.vertices.filter(_._1 >= 0).map { case (docID, topicCounts) => - (docID.toLong, Vectors.fromBreeze(topicCounts)) - } - } - - // TODO: - // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? - - /* - // TODO: Do this properly - lazy val logLikelihood = { - graph.triplets.aggregate(0.0)({ (acc, triple) => - val scores = triple.srcAttr :* triple.dstAttr - val logScores = breeze.numerics.log(scores) - scores /= brzSum(scores) - brzSum(scores :*= logScores) * triple.attr - }, _ + _) - } - */ - - /** - * Return the topics described by the top-weighted terms. - * - * Note: This is approximate; it may not return exactly the top-weighted terms for each topic. - * To get a more precise set of top terms, increase maxTermsPerTopic. - * - * @param maxTermsPerTopic Maximum number of terms to collect for each topic. - * @return Array over topics, where each element is a set of top terms represented - * as (term weight in topic, term index). - * Each topic's terms are sorted in order of decreasing weight. - */ - def getTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { - val numTopics = k - val N_k: TopicCounts = state.collectTopicTotals() - state.graph.vertices.filter(isTermVertex) - .mapPartitions { termVertices => - // For this partition, collect the most common terms for each topic in queues: - // queues(topic) = queue of (term weight, term index). - // Term weights are N_{wk} / N_k. - val queues = Array.fill(numTopics)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) - for ((termId, n_wk) <- termVertices) { - var topic = 0 - while (topic < numTopics) { - queues(topic) += (n_wk(topic) / N_k(topic) -> index2term(termId.toInt)) - topic += 1 - } - } - Iterator(queues) - }.reduce { (q1, q2) => - q1.zip(q2).foreach { case (a, b) => a ++= b} - q1 - }.map(_.toArray.sortBy(_._1)) - } - -} object LDA { @@ -356,7 +187,7 @@ object LDA { - Edges are partitioned by documents. Info on EM implementation. - - We follow Section 2.2 from Asuncion et al., 2009. + - We follow Section 2.2 from Asuncion et al., 2009. We use some of their notation. - In this implementation, there is one edge for every unique term appearing in a document, i.e., for every unique (document, term) pair. - Notation: @@ -387,7 +218,9 @@ object LDA { */ /** - * Document + * :: DeveloperApi :: + * + * Document with an ID. * * @param counts Vector of term (word) counts in the document. * This is the "bag of words" representation. @@ -395,8 +228,12 @@ object LDA { * Documents should be indexed {0, 1, ..., numDocuments-1}. * * TODO: Can we remove the id and still be able to zip predicted topics with the Documents? + * + * NOTE: This is currently marked DeveloperApi since it is under active development and may + * undergo API changes. */ - case class Document(counts: SparseVector, id: VertexId) + @DeveloperApi + case class Document(counts: SparseVector, id: Long) /** * Vector over topics (length k) of token counts. @@ -431,14 +268,13 @@ object LDA { termSmoothing: Double) { // TODO: Checkpoint periodically? - def next() = copy(graph = step(graph)) + def next(): LearningState = copy(graph = step(graph)) private def step(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { val eta = termSmoothing val W = vocabSize val alpha = topicSmoothing - // Collect N_k from term vertices. val N_k = collectTopicTotals() val sendMsg: EdgeContext[TopicCounts, TokenCount, TopicCounts] => Unit = (edgeContext) => { // Compute N_{wj} gamma_{wjk} @@ -456,10 +292,74 @@ object LDA { graph.outerJoinVertices(docTopicDistributions){ (vid, oldDist, newDist) => newDist.get } } - private[clustering] def collectTopicTotals(): TopicCounts = { + def collectTopicTotals(): TopicCounts = { val numTopics = k graph.vertices.filter(isTermVertex).map(_._2).fold(BDV.zeros[Double](numTopics))(_ + _) } + + /** + * Compute the log likelihood of the observed tokens, given the current parameter estimates: + * log P(docs | topics, topic distributions for docs, alpha, eta) + * + * Note: + * - This excludes the prior; for that, use [[logPrior]]. + * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the + * hyperparameters. + */ + lazy val logLikelihood: Double = { + val eta = termSmoothing + val alpha = topicSmoothing + assert(eta > 1.0) + assert(alpha > 1.0) + val N_k = collectTopicTotals() + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + // Edges: Compute token log probability from phi_{wk}, theta_{kj}. + val sendMsg: EdgeContext[TopicCounts, TokenCount, Double] => Unit = (edgeContext) => { + val N_wj = edgeContext.attr + val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) + val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) + val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) + val tokenLogLikelihood = N_wj * math.log(phi_wk.dot(theta_kj)) + edgeContext.sendToDst(tokenLogLikelihood) + } + graph.aggregateMessages[Double](sendMsg, _ + _) + .map(_._2).fold(0.0)(_ + _) + } + + /** + * Compute the log probability of the current parameter estimate: + * log P(topics, topic distributions for docs | alpha, eta) + */ + lazy val logPrior: Double = { + val eta = termSmoothing + val alpha = topicSmoothing + // Term vertices: Compute phi_{wk}. Use to compute prior log probability. + // Doc vertex: Compute theta_{kj}. Use to compute prior log probability. + val N_k = collectTopicTotals() + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + val seqOp: (Double, (VertexId, TopicCounts)) => Double = { + case (sumPrior: Double, vertex: (VertexId, TopicCounts)) => + if (isTermVertex(vertex)) { + val N_wk = vertex._2 + val smoothed_N_wk: TopicCounts = N_wk + (eta - 1.0) + val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + if (!phi_wk.forall(_ > 0)) { + println(s"ERROR: phi_wk = ${phi_wk.toArray.mkString(", ")}") + } + (eta - 1.0) * brzSum(phi_wk.map(math.log)) + } else { + val N_kj = vertex._2 + val smoothed_N_kj: TopicCounts = N_kj + (alpha - 1.0) + val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) + if (!theta_kj.forall(_ > 0)) { + println(s"ERROR: theta_kj = ${theta_kj.toArray.mkString(", ")}") + } + (alpha - 1.0) * brzSum(theta_kj.map(math.log)) + } + } + graph.vertices.aggregate(0.0)(seqOp, _ + _) + } } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala new file mode 100644 index 0000000000000..85b73bf822d60 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -0,0 +1,265 @@ +/* + * 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.mllib.clustering + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.BoundedPriorityQueue + +/** + * :: DeveloperApi :: + * + * Latent Dirichlet Allocation (LDA) model. + * + * This abstraction permits for different underlying representations, + * including local and distributed data structures. + */ +@DeveloperApi +abstract class LDAModel private[clustering] { + + import LDA._ + + /** Number of topics */ + def k: Int + + /** Vocabulary size (number of terms or terms in the vocabulary) */ + def vocabSize: Int + + /** + * 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. + */ + def topicsMatrix: Matrix + + /** + * Return the topics described by weighted terms. + * + * This limits the number of terms per topic. + * This is approximate; it may not return exactly the top-weighted terms for each topic. + * To get a more precise set of top terms, increase maxTermsPerTopic. + * + * @param maxTermsPerTopic Maximum number of terms to collect for each topic. + * @return Array over topics, where each element is a set of top terms represented + * as (term weight in topic, term index). + * Each topic's terms are sorted in order of decreasing weight. + */ + def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] + + /** + * Return the topics described by weighted terms. + * + * WARNING: If vocabSize and k are large, this can return a large object! + * + * @return Array over topics, where each element is a set of top terms represented + * as (term weight in topic, term index). + * Each topic's terms are sorted in order of decreasing weight. + */ + def describeTopics(): Array[Array[(Double, String)]] = describeTopics(vocabSize) + + /* TODO + * Compute the log likelihood of the observed tokens, given the current parameter estimates: + * log P(docs | topics, topic distributions for docs, alpha, eta) + * + * Note: + * - This excludes the prior. + * - Even with the prior, this is NOT the same as the data log likelihood given the + * hyperparameters. + * + * @param documents A set of documents, where each is represented as a vector of term counts. + * This must use the same vocabulary (ordering of term counts) as in training. + * Document IDs must be >= 0. + * @return Estimated log likelihood of the data under this model + */ + //def logLikelihood(documents: RDD[Document]): Double + + /* TODO + * Compute the estimated topic distribution for each document. + * This is often called “theta” in the literature. + * + * @param documents A set of documents, where each is represented as a vector of term counts. + * This must use the same vocabulary (ordering of term counts) as in training. + * Document IDs must be >= 0. + * @return Estimated topic distribution for each document. + * The returned RDD may be zipped with the given RDD, where each returned vector + * is a multinomial distribution over topics. + */ + // def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] + +} + +/** + * :: DeveloperApi :: + * + * Local LDA model. + * This model stores only the inferred topics. + * It may be used for computing topics for new documents, but it may give less accurate answers + * than the [[DistributedLDAModel]]. + * + * NOTE: This is currently marked DeveloperApi since it is under active development and may undergo + * API changes. + * + * @param topics Inferred topics (vocabSize x k matrix). + */ +@DeveloperApi +class LocalLDAModel private[clustering] ( + private val topics: Matrix) extends LDAModel with Serializable { + + import LDA._ + + override def k: Int = topics.numCols + + override def vocabSize: Int = topics.numRows + + override def topicsMatrix: Matrix = topics + + override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] = { + val brzTopics = topics.toBreeze.toDenseMatrix + val topicSummary = Range(0, k).map { topicIndex => + val topic: Array[Double] = brzTopics(::, topicIndex).toArray + topic.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic) + }.toArray + topicSummary.map { topic => + topic.map { case (weight, term) => (weight, term.toString) } + } + } + + // TODO + //override def logLikelihood(documents: RDD[Document]): Double = ??? + + // TODO: + // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? + +} + +/** + * :: DeveloperApi :: + * + * Distributed LDA model. + * This model stores the inferred topics, the full training dataset, and the topic distributions. + * When computing topics for new documents, it may give more accurate answers + * than the [[LocalLDAModel]]. + * + * NOTE: This is currently marked DeveloperApi since it is under active development and may undergo + * API changes. + */ +@DeveloperApi +class DistributedLDAModel private[clustering] ( + private val state: LDA.LearningState) extends LDAModel { + + import LDA._ + + def toLocal: LocalLDAModel = new LocalLDAModel(topicsMatrix) + + override def k: Int = state.k + + override def vocabSize: Int = state.vocabSize + + /** + * 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: This matrix is collected from an RDD. Beware memory usage when vocabSize, k are large. + */ + override lazy val topicsMatrix: Matrix = { + // Collect row-major topics + val termTopicCounts: Array[(Int, TopicCounts)] = + state.graph.vertices.filter(_._1 < 0).map { case (termIndex, cnts) => + (index2term(termIndex), cnts) + }.collect() + // Convert to Matrix + val brzTopics = BDM.zeros[Double](vocabSize, k) + termTopicCounts.foreach { case (term, cnts) => + var j = 0 + while (j < k) { + brzTopics(term, j) = cnts(j) + j += 1 + } + } + Matrices.fromBreeze(brzTopics) + } + + override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] = { + val numTopics = k + // Note: N_k is not needed to find the top terms, but it is needed to normalize weights + // to a distribution over terms. + val N_k: TopicCounts = state.collectTopicTotals() + val topicSummary = state.graph.vertices.filter(isTermVertex) + .mapPartitions { termVertices => + // For this partition, collect the most common terms for each topic in queues: + // queues(topic) = queue of (term weight, term index). + // Term weights are N_{wk} / N_k. + val queues = Array.fill(numTopics)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) + for ((termId, n_wk) <- termVertices) { + var topic = 0 + while (topic < numTopics) { + queues(topic) += (n_wk(topic) / N_k(topic) -> index2term(termId.toInt)) + topic += 1 + } + } + Iterator(queues) + }.reduce { (q1, q2) => + q1.zip(q2).foreach { case (a, b) => a ++= b} + q1 + }.map(_.toArray.sortBy(-_._1)) + topicSummary.map { topic => + topic.map { case (weight, term) => (weight, term.toString) } + } + } + + // TODO + //override def logLikelihood(documents: RDD[Document]): Double = ??? + + /** + * Compute the log likelihood of the observed tokens in the training set, + * given the current parameter estimates: + * log P(docs | topics, topic distributions for docs, alpha, eta) + * + * Note: + * - This excludes the prior; for that, use [[logPrior]]. + * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the + * hyperparameters. + */ + def logLikelihood = state.logLikelihood + + /** + * Compute the log probability of the current parameter estimate, under the prior: + * log P(topics, topic distributions for docs | alpha, eta) + */ + def logPrior = state.logPrior + + /** + * For each document in the training set, return the distribution over topics for that document + * (i.e., "theta_doc"). + * + * @return RDD of (document ID, topic distribution) pairs + */ + def topicDistributions: RDD[(Long, Vector)] = { + state.graph.vertices.filter(_._1 >= 0).map { case (docID, topicCounts) => + (docID.toLong, Vectors.fromBreeze(topicCounts)) + } + } + + // TODO: + // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? + +} From 9f2a4929973353c064912bbe106911b31e8bc05f Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 14 Jan 2015 10:17:20 -0800 Subject: [PATCH 05/30] Unit tests and fixes for LDA, now ready for PR --- .../apache/spark/mllib/clustering/LDA.scala | 67 ++++---- .../spark/mllib/clustering/LDAModel.scala | 10 +- .../spark/mllib/clustering/LDASuite.scala | 143 ++++++++++++++++++ 3 files changed, 185 insertions(+), 35 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index b8dc6345fdc8d..a345108728a3e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -21,9 +21,10 @@ import java.util.Random import breeze.linalg.{DenseVector => BDV, sum => brzSum, normalize} +import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.graphx._ -import org.apache.spark.mllib.linalg.SparseVector +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -58,11 +59,11 @@ class LDA private ( private var maxIterations: Int, private var topicSmoothing: Double, private var termSmoothing: Double, - private var seed: Long) { + private var seed: Long) extends Logging { import LDA._ - def this() = this(k = 10, maxIterations = 10, topicSmoothing = -1, termSmoothing = 0.1, + def this() = this(k = 10, maxIterations = 20, topicSmoothing = -1, termSmoothing = -1, seed = Utils.random.nextLong()) /** @@ -84,19 +85,22 @@ class LDA private ( * ("theta"). We use a symmetric Dirichlet prior. * * This value should be > 0.0, where larger values mean more smoothing (more regularization). - * If set to -1, then topicSmoothing is set to equal 50 / k (where k is the number of topics). - * (default = 50 / k) + * If set to -1, then topicSmoothing is set automatically. + * (default = -1 = automatic) * - * Details on this parameter: - * The above description tells how the base value of topicSmoothing is set. However, following - * Asuncion et al. (2009), we adjust the base value based on the algorithm. - * - For EM, we increase topicSmoothing by +1.0. + * Automatic setting of parameter: + * - For EM: default = (50 / k) + 1. + * - The 50/k is common in LDA libraries. + * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. */ def getTopicSmoothing: Double = topicSmoothing def setTopicSmoothing(topicSmoothing: Double): this.type = { require(topicSmoothing > 0.0 || topicSmoothing == -1.0, - s"LDA topicSmoothing must be > 0, but was set to $topicSmoothing") + s"LDA topicSmoothing must be > 0 (or -1 for auto), but was set to $topicSmoothing") + if (topicSmoothing > 0.0 && topicSmoothing <= 1.0) { + logWarning(s"LDA.topicSmoothing was set to $topicSmoothing, but for EM, we recommend > 1.0") + } this.topicSmoothing = topicSmoothing this } @@ -109,24 +113,29 @@ class LDA private ( * later papers such as Asuncion et al., 2009.) * * This value should be > 0.0. - * (default = 0.1) + * If set to -1, then termSmoothing is set automatically. + * (default = -1 = automatic) * - * Details on this parameter: - * The above description tells how the base value of topicSmoothing is set. However, following - * Asuncion et al. (2009), we adjust the base value based on the algorithm. - * - For EM, we increase topicSmoothing by +1.0. + * Automatic setting of parameter: + * - For EM: default = 0.1 + 1. + * - The 0.1 gives a small amount of smoothing. + * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. */ def getTermSmoothing: Double = termSmoothing def setTermSmoothing(termSmoothing: Double): this.type = { - require(termSmoothing > 0.0, s"LDA termSmoothing must be > 0, but was set to $termSmoothing") + require(termSmoothing > 0.0 || termSmoothing == -1.0, + s"LDA termSmoothing must be > 0 (or -1 for auto), but was set to $termSmoothing") + if (termSmoothing > 0.0 && termSmoothing <= 1.0) { + logWarning(s"LDA.termSmoothing was set to $termSmoothing, but for EM, we recommend > 1.0") + } this.termSmoothing = termSmoothing this } /** * Maximum number of iterations for learning. - * (default = 10) + * (default = 20) */ def getMaxIterations: Int = maxIterations @@ -151,18 +160,20 @@ class LDA private ( * @return Inferred LDA model */ def run(documents: RDD[Document]): DistributedLDAModel = { - val termSmoothing = this.termSmoothing + 1.0 - val topicSmoothing = (if (this.topicSmoothing > 0) { + val topicSmoothing = if (this.topicSmoothing > 0) { this.topicSmoothing } else { - 50.0 / k - }) + 1.0 + (50.0 / k) + 1.0 + } + val termSmoothing = if (this.termSmoothing > 0) { + this.termSmoothing + } else { + 1.1 + } var state = LDA.initialState(documents, k, topicSmoothing, termSmoothing, seed) - println(s"DEBUG: initial log likelihood = ${state.logLikelihood}") var iter = 0 while (iter < maxIterations) { state = state.next() - println(s"DEBUG: iter=$iter. log likelihood = ${state.logLikelihood}") iter += 1 } new DistributedLDAModel(state) @@ -233,7 +244,7 @@ object LDA { * undergo API changes. */ @DeveloperApi - case class Document(counts: SparseVector, id: Long) + case class Document(counts: Vector, id: Long) /** * Vector over topics (length k) of token counts. @@ -248,6 +259,8 @@ object LDA { private[clustering] def index2term(termIndex: Long): Int = -(1 + termIndex).toInt + private[clustering] def isDocumentVertex(v: (VertexId, _)): Boolean = v._1 >= 0 + private[clustering] def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 /** @@ -344,17 +357,11 @@ object LDA { val N_wk = vertex._2 val smoothed_N_wk: TopicCounts = N_wk + (eta - 1.0) val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k - if (!phi_wk.forall(_ > 0)) { - println(s"ERROR: phi_wk = ${phi_wk.toArray.mkString(", ")}") - } (eta - 1.0) * brzSum(phi_wk.map(math.log)) } else { val N_kj = vertex._2 val smoothed_N_kj: TopicCounts = N_kj + (alpha - 1.0) val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) - if (!theta_kj.forall(_ > 0)) { - println(s"ERROR: theta_kj = ${theta_kj.toArray.mkString(", ")}") - } (alpha - 1.0) * brzSum(theta_kj.map(math.log)) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 85b73bf822d60..cc446c346ef7e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.clustering -import breeze.linalg.{DenseMatrix => BDM} +import breeze.linalg.{DenseMatrix => BDM, normalize} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} @@ -134,8 +134,8 @@ class LocalLDAModel private[clustering] ( override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] = { val brzTopics = topics.toBreeze.toDenseMatrix val topicSummary = Range(0, k).map { topicIndex => - val topic: Array[Double] = brzTopics(::, topicIndex).toArray - topic.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic) + val topic = normalize(brzTopics(::, topicIndex), 1.0) + topic.toArray.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic) }.toArray topicSummary.map { topic => topic.map { case (weight, term) => (weight, term.toString) } @@ -254,8 +254,8 @@ class DistributedLDAModel private[clustering] ( * @return RDD of (document ID, topic distribution) pairs */ def topicDistributions: RDD[(Long, Vector)] = { - state.graph.vertices.filter(_._1 >= 0).map { case (docID, topicCounts) => - (docID.toLong, Vectors.fromBreeze(topicCounts)) + state.graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => + (docID.toLong, Vectors.fromBreeze(normalize(topicCounts, 1.0))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala new file mode 100644 index 0000000000000..1aaabe44a6be8 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -0,0 +1,143 @@ +/* + * 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.mllib.clustering + +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.scalatest.FunSuite + +class LDASuite extends FunSuite with MLlibTestSparkContext { + + import LDASuite._ + + test("LocalLDAModel") { + val model = new LocalLDAModel(tinyTopics) + + // Check: basic parameters + assert(model.k === tinyK) + assert(model.vocabSize === tinyVocabSize) + assert(model.topicsMatrix === tinyTopics) + + // Check: describeTopics() with all terms + val fullTopicSummary = model.describeTopics() + assert(fullTopicSummary.size === tinyK) + fullTopicSummary.zip(tinyTopicDescription).foreach { case (algSummary, tinySummary) => + assert(algSummary === tinySummary) + } + + // Check: describeTopics() with some terms + val smallNumTerms = 3 + val smallTopicSummary = model.describeTopics(maxTermsPerTopic = smallNumTerms) + smallTopicSummary.zip(tinyTopicDescription).foreach { case (algSummary, tinySummary) => + assert(algSummary === tinySummary.slice(0, smallNumTerms)) + } + } + + test("running and DistributedLDAModel") { + val k = 3 + val topicSmoothing = 1.2 + val termSmoothing = 1.2 + + // Train a model + val lda = new LDA() + lda.setK(k) + lda.setTopicSmoothing(topicSmoothing) + lda.setTermSmoothing(termSmoothing) + lda.setMaxIterations(5) + lda.setSeed(12345) + val corpus = sc.parallelize(tinyCorpus, 2) + + val model: DistributedLDAModel = lda.run(corpus) + + // Check: basic parameters + val localModel = model.toLocal + assert(model.k === k) + assert(localModel.k === k) + assert(model.vocabSize === tinyVocabSize) + assert(localModel.vocabSize === tinyVocabSize) + assert(model.topicsMatrix === localModel.topicsMatrix) + + // Check: topic summaries + // The odd decimal formatting and sorting is a hack to do a robust comparison. + val roundedTopicSummary = model.describeTopics().map { case topic => + // cut values to 3 digits after the decimal place + topic.map { case (weight, term) => ("%.3f".format(weight).toDouble, term.toInt)} + }.sortBy(_.mkString("")) + val roundedLocalTopicSummary = localModel.describeTopics().map { case topic => + // cut values to 3 digits after the decimal place + topic.map { case (weight, term) => ("%.3f".format(weight).toDouble, term.toInt)} + }.sortBy(_.mkString("")) + roundedTopicSummary.zip(roundedLocalTopicSummary).foreach { case (t1, t2) => + assert(t1 === t2) + } + + // Check: per-doc topic distributions + val topicDistributions = model.topicDistributions.collect() + // Ensure all documents are covered. + assert(topicDistributions.size === tinyCorpus.size) + assert(tinyCorpus.map(_.id).toSet === topicDistributions.map(_._1).toSet) + // Ensure we have proper distributions + topicDistributions.foreach { case (docId, topicDistribution) => + assert(topicDistribution.size === tinyK) + assert(topicDistribution.toArray.sum ~== 1.0 absTol 1e-5) + } + + // Check: log probabilities + assert(model.logLikelihood < 0.0) + assert(model.logPrior < 0.0) + } + + test("vertex indexing") { + // Check vertex ID indexing and conversions. + val docIds = Array(0, 1, 2) + val docVertexIds = docIds + val termIds = Array(0, 1, 2) + val termVertexIds = Array(-1, -2, -3) + assert(docVertexIds.forall(i => !LDA.isTermVertex((i.toLong, 0)))) + assert(termIds.map(LDA.term2index) === termVertexIds) + assert(termVertexIds.map(i => LDA.index2term(i.toLong)) === termIds) + assert(termVertexIds.forall(i => LDA.isTermVertex((i.toLong, 0)))) + } +} + +private object LDASuite { + + def tinyK: Int = 3 + def tinyVocabSize: Int = 5 + def tinyTopicsAsArray: Array[Array[Double]] = Array( + Array[Double](0.1, 0.2, 0.3, 0.4, 0.0), // topic 0 + Array[Double](0.5, 0.05, 0.05, 0.1, 0.3), // topic 1 + Array[Double](0.2, 0.2, 0.05, 0.05, 0.5) // topic 2 + ) + def tinyTopics: Matrix = new DenseMatrix(numRows = tinyVocabSize, numCols = tinyK, + values = tinyTopicsAsArray.fold(Array.empty[Double])(_ ++ _)) + def tinyTopicDescription: Array[Array[(Double, String)]] = tinyTopicsAsArray.map { topic => + topic.zipWithIndex.sortBy(-_._1).map { case (weight, term) => (weight, term.toString) } + } + + def tinyCorpus = Array( + Vectors.dense(1, 3, 0, 2, 8), + Vectors.dense(0, 2, 1, 0, 4), + Vectors.dense(2, 3, 12, 3, 1), + Vectors.dense(0, 3, 1, 9, 8), + Vectors.dense(1, 1, 4, 2, 6) + ).zipWithIndex.map { case (wordCounts, docId) => LDA.Document(wordCounts, docId) } + assert(tinyCorpus.forall(_.counts.size == tinyVocabSize)) // sanity check for test data + +} From 75749e7b13a6a9b83e83ab2605fa2af714c85bb4 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 14 Jan 2015 11:20:21 -0800 Subject: [PATCH 06/30] scala style fix --- .../scala/org/apache/spark/mllib/clustering/LDAModel.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index cc446c346ef7e..66b3ded6fa5d8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -89,7 +89,7 @@ abstract class LDAModel private[clustering] { * Document IDs must be >= 0. * @return Estimated log likelihood of the data under this model */ - //def logLikelihood(documents: RDD[Document]): Double + // def logLikelihood(documents: RDD[Document]): Double /* TODO * Compute the estimated topic distribution for each document. @@ -143,7 +143,7 @@ class LocalLDAModel private[clustering] ( } // TODO - //override def logLikelihood(documents: RDD[Document]): Double = ??? + // override def logLikelihood(documents: RDD[Document]): Double = ??? // TODO: // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? @@ -227,7 +227,7 @@ class DistributedLDAModel private[clustering] ( } // TODO - //override def logLikelihood(documents: RDD[Document]): Double = ??? + // override def logLikelihood(documents: RDD[Document]): Double = ??? /** * Compute the log likelihood of the observed tokens in the training set, From ce53be9c32aec354027c2aa298b37098ca043329 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 14 Jan 2015 11:37:29 -0800 Subject: [PATCH 07/30] fixed example name --- .../main/scala/org/apache/spark/examples/mllib/LDAExample.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index ac0f48d0b45d1..f18f532509768 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -36,7 +36,7 @@ import org.apache.spark.rdd.RDD /** * An example Latent Dirichlet Allocation (LDA) app. Run with * {{{ - * ./bin/run-example mllib.DenseKMeans [options] + * ./bin/run-example mllib.LDAExample [options] * }}} * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ From 45cc7f2d172d1d16c3369d2981c8216e0dcbf80d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 13:17:47 -0800 Subject: [PATCH 08/30] mapPart -> flatMap --- .../scala/org/apache/spark/mllib/clustering/LDA.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index a345108728a3e..e750f1b7d32d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -398,12 +398,10 @@ object LDA { termSmoothing: Double, randomSeed: Long): LearningState = { // For each document, create an edge (Document -> Term) for each unique term in the document. - val edges: RDD[Edge[TokenCount]] = docs.mapPartitionsWithIndex { case (partIndex, partDocs) => - partDocs.flatMap { doc: Document => - // Add edges for terms with non-zero counts. - doc.counts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => - Edge(doc.id, term2index(term), cnt) - } + val edges: RDD[Edge[TokenCount]] = docs.flatMap { case doc => + // Add edges for terms with non-zero counts. + doc.counts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + Edge(doc.id, term2index(term), cnt) } } @@ -436,5 +434,4 @@ object LDA { LearningState(graph, k, vocabSize, topicSmoothing, termSmoothing) } - } From 892530ce765128b840eeed5d0ff0a52810c6e5f1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 13:34:05 -0800 Subject: [PATCH 09/30] use axpy --- .../org/apache/spark/mllib/clustering/LDA.scala | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index e750f1b7d32d4..2a12cd03fea97 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import java.util.Random -import breeze.linalg.{DenseVector => BDV, sum => brzSum, normalize} +import breeze.linalg.{DenseVector => BDV, sum => brzSum, normalize, axpy => brzAxpy} import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi @@ -418,12 +418,19 @@ object LDA { } } def createVertices(sendToWhere: Edge[TokenCount] => VertexId): RDD[(VertexId, TopicCounts)] = { - val verticesTMP: RDD[(VertexId, TopicCounts)] = + val verticesTMP: RDD[(VertexId, (TokenCount, TopicCounts))] = edgesWithGamma.map { case (edge, gamma: TopicCounts) => - val N_wj = edge.attr - (sendToWhere(edge), gamma * N_wj) + (sendToWhere(edge), (edge.attr, gamma)) } - verticesTMP.foldByKey(BDV.zeros[Double](k))(_ + _) + verticesTMP.aggregateByKey(BDV.zeros[Double](k))( + (sum, t) => { + brzAxpy(t._1, t._2, sum) + sum + }, + (sum0, sum1) => { + sum0 += sum1 + } + ) } val docVertices = createVertices(_.srcId) val termVertices = createVertices(_.dstId) From 9eb3d02ad9bfeee3fce22d2cb3b5a5578ee21fb9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 13:36:53 -0800 Subject: [PATCH 10/30] + -> += --- .../src/main/scala/org/apache/spark/mllib/clustering/LDA.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 2a12cd03fea97..7d65e89c376a1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -307,7 +307,7 @@ object LDA { def collectTopicTotals(): TopicCounts = { val numTopics = k - graph.vertices.filter(isTermVertex).map(_._2).fold(BDV.zeros[Double](numTopics))(_ + _) + graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) } /** From 6cb11b0d1428bc1c649f3d0a8d55c8b8a1c67e8c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 14:20:15 -0800 Subject: [PATCH 11/30] optimize computePTopic --- .../apache/spark/mllib/clustering/LDA.scala | 23 ++++++++++++++----- 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 7d65e89c376a1..39aabc986ab00 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -378,13 +378,24 @@ object LDA { vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { - val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) - val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) - val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) - // proportional to p(w|z) * p(z|d) / p(z) - val unnormalizedGamma = smoothed_N_wk :* smoothed_N_kj :/ smoothed_N_k + val N = N_k.data + val N_w = edgeContext.dstAttr.data + val N_j = edgeContext.srcAttr.data + val K = N_w.size + val eta1 = eta - 1.0 + val alpha1 = alpha - 1.0 + val Weta1 = vocabSize * eta1 + var sum = 0.0 + val gamma_wj = new Array[Double](K) + var k = 0 + while (k < K) { + val gamma_wjk = (N_w(k) + eta1) * (N_j(k) + alpha1) / (N(k) + Weta1) + gamma_wj(k) = gamma_wjk + sum += gamma_wjk + k += 1 + } // normalize - unnormalizedGamma /= brzSum(unnormalizedGamma) + BDV(gamma_wj) /= sum } /** From cec0a9c19f6ea846272e4fa0a3d0121e45b9c536 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 14:40:22 -0800 Subject: [PATCH 12/30] * -> *= --- .../src/main/scala/org/apache/spark/mllib/clustering/LDA.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 39aabc986ab00..4b61a69df53a1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -294,7 +294,7 @@ object LDA { val N_wj = edgeContext.attr // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count N_{wj}. val scaledTopicDistribution: TopicCounts = - computePTopic(edgeContext, N_k, W, eta, alpha) * N_wj + computePTopic(edgeContext, N_k, W, eta, alpha) *= N_wj edgeContext.sendToDst(scaledTopicDistribution) edgeContext.sendToSrc(scaledTopicDistribution) } From 9fe0b956bcd102ac6ed5e587078c49f5bacd373f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 15:10:43 -0800 Subject: [PATCH 13/30] optimize aggregateMessages --- .../apache/spark/mllib/clustering/LDA.scala | 74 ++++++++++++------- 1 file changed, 46 insertions(+), 28 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 4b61a69df53a1..6815c9512a640 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -279,7 +279,6 @@ object LDA { vocabSize: Int, topicSmoothing: Double, termSmoothing: Double) { - // TODO: Checkpoint periodically? def next(): LearningState = copy(graph = step(graph)) @@ -289,20 +288,38 @@ object LDA { val alpha = topicSmoothing val N_k = collectTopicTotals() - val sendMsg: EdgeContext[TopicCounts, TokenCount, TopicCounts] => Unit = (edgeContext) => { - // Compute N_{wj} gamma_{wjk} - val N_wj = edgeContext.attr - // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count N_{wj}. - val scaledTopicDistribution: TopicCounts = - computePTopic(edgeContext, N_k, W, eta, alpha) *= N_wj - edgeContext.sendToDst(scaledTopicDistribution) - edgeContext.sendToSrc(scaledTopicDistribution) - } + val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = + (edgeContext) => { + // Compute N_{wj} gamma_{wjk} + val N_wj = edgeContext.attr + // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count + // N_{wj}. + val scaledTopicDistribution: TopicCounts = + computePTopic(edgeContext.srcAttr, edgeContext.dstAttr, N_k, W, eta, alpha) *= N_wj + edgeContext.sendToDst((false, scaledTopicDistribution)) + edgeContext.sendToSrc((false, scaledTopicDistribution)) + } + // This is a hack to detect whether we could modify the values in-place. + // TODO: Add zero/seqOp/combOp option to aggregateMessages. + val mergeMsg: ((Boolean, TopicCounts), (Boolean, TopicCounts)) => (Boolean, TopicCounts) = + (m0, m1) => { + val sum = + if (m0._1) { + m0._2 += m1._2 + } else if (m1._1) { + m1._2 += m0._2 + } else { + val k = m0._2.length + m0._2 + m1._2 + } + (true, sum) + } // M-STEP: Aggregation computes new N_{kj}, N_{wk} counts. val docTopicDistributions: VertexRDD[TopicCounts] = - graph.aggregateMessages[TopicCounts](sendMsg, _ + _) + graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) + .mapValues(_._2) // Update the vertex descriptors with the new counts. - graph.outerJoinVertices(docTopicDistributions){ (vid, oldDist, newDist) => newDist.get } + graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get} } def collectTopicTotals(): TopicCounts = { @@ -312,12 +329,12 @@ object LDA { /** * Compute the log likelihood of the observed tokens, given the current parameter estimates: - * log P(docs | topics, topic distributions for docs, alpha, eta) + * log P(docs | topics, topic distributions for docs, alpha, eta) * * Note: - * - 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 excludes the prior; for that, use [[logPrior]]. + * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the + * hyperparameters. */ lazy val logLikelihood: Double = { val eta = termSmoothing @@ -342,7 +359,7 @@ object LDA { /** * Compute the log probability of the current parameter estimate: - * log P(topics, topic distributions for docs | alpha, eta) + * log P(topics, topic distributions for docs | alpha, eta) */ lazy val logPrior: Double = { val eta = termSmoothing @@ -373,15 +390,16 @@ object LDA { * Compute gamma_{wjk}, a distribution over topics k. */ private def computePTopic( - edgeContext: EdgeContext[TopicCounts, TokenCount, TopicCounts], - N_k: TopicCounts, + docTopicCounts: TopicCounts, + wordTopicCounts: TopicCounts, + totalTopicCounts: TopicCounts, vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { - val N = N_k.data - val N_w = edgeContext.dstAttr.data - val N_j = edgeContext.srcAttr.data - val K = N_w.size + val K = docTopicCounts.length + val N_j = docTopicCounts.data + val N_w = wordTopicCounts.data + val N = totalTopicCounts.data val eta1 = eta - 1.0 val alpha1 = alpha - 1.0 val Weta1 = vocabSize * eta1 @@ -398,11 +416,11 @@ object LDA { BDV(gamma_wj) /= sum } - /** - * Compute bipartite term/doc graph. - * doc ids are shifted by vocabSize to maintain uniqueness - */ - private def initialState( + /** + * Compute bipartite term/doc graph. + * doc ids are shifted by vocabSize to maintain uniqueness + */ + private def initialState( docs: RDD[Document], k: Int, topicSmoothing: Double, From 08d59a380ac9aaac120e5011dc916ec5d887508c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 15:14:03 -0800 Subject: [PATCH 14/30] reset spacing --- .../apache/spark/mllib/clustering/LDA.scala | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 6815c9512a640..38aee42f34e75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -329,12 +329,12 @@ object LDA { /** * Compute the log likelihood of the observed tokens, given the current parameter estimates: - * log P(docs | topics, topic distributions for docs, alpha, eta) + * log P(docs | topics, topic distributions for docs, alpha, eta) * * Note: - * - 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 excludes the prior; for that, use [[logPrior]]. + * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the + * hyperparameters. */ lazy val logLikelihood: Double = { val eta = termSmoothing @@ -359,7 +359,7 @@ object LDA { /** * Compute the log probability of the current parameter estimate: - * log P(topics, topic distributions for docs | alpha, eta) + * log P(topics, topic distributions for docs | alpha, eta) */ lazy val logPrior: Double = { val eta = termSmoothing @@ -416,11 +416,11 @@ object LDA { BDV(gamma_wj) /= sum } - /** - * Compute bipartite term/doc graph. - * doc ids are shifted by vocabSize to maintain uniqueness - */ - private def initialState( + /** + * Compute bipartite term/doc graph. + * doc ids are shifted by vocabSize to maintain uniqueness + */ + private def initialState( docs: RDD[Document], k: Int, topicSmoothing: Double, From fb1e7b5848ca6c4d172328449ca723c2123c7eee Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 16 Jan 2015 15:16:02 -0800 Subject: [PATCH 15/30] minor --- .../main/scala/org/apache/spark/mllib/clustering/LDA.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 38aee42f34e75..b4a80a16fa76f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -309,7 +309,6 @@ object LDA { } else if (m1._1) { m1._2 += m0._2 } else { - val k = m0._2.length m0._2 + m1._2 } (true, sum) @@ -319,7 +318,7 @@ object LDA { graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) .mapValues(_._2) // Update the vertex descriptors with the new counts. - graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get} + graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get } } def collectTopicTotals(): TopicCounts = { @@ -427,7 +426,7 @@ object LDA { termSmoothing: Double, randomSeed: Long): LearningState = { // For each document, create an edge (Document -> Term) for each unique term in the document. - val edges: RDD[Edge[TokenCount]] = docs.flatMap { case doc => + val edges: RDD[Edge[TokenCount]] = docs.flatMap { doc => // Add edges for terms with non-zero counts. doc.counts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => Edge(doc.id, term2index(term), cnt) From 77a2c8579b43fbe4abe8dac5fecad0a20daa8d8e Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 21 Jan 2015 13:46:32 -0800 Subject: [PATCH 16/30] Moved auto term,topic smoothing computation to get*Smoothing methods. Changed word to term in some places. Updated LDAExample to use default smoothing amounts. --- .../spark/examples/mllib/LDAExample.scala | 10 +++--- .../apache/spark/mllib/clustering/LDA.scala | 34 ++++++++++--------- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index f18f532509768..718bbaffab73f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -46,8 +46,8 @@ object LDAExample { input: Seq[String] = Seq.empty, k: Int = 20, maxIterations: Int = 10, - topicSmoothing: Double = 0.1, - termSmoothing: Double = 0.1, + topicSmoothing: Double = -1, + termSmoothing: Double = -1, vocabSize: Int = 10000, stopwordFile: String = "") extends AbstractParams[Params] @@ -63,10 +63,10 @@ object LDAExample { .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) opt[Double]("topicSmoothing") - .text(s"amount of topic smoothing to use. default: ${defaultParams.topicSmoothing}") + .text(s"amount of topic smoothing to use (-1=auto). default: ${defaultParams.topicSmoothing}") .action((x, c) => c.copy(topicSmoothing = x)) opt[Double]("termSmoothing") - .text(s"amount of word smoothing to use. default: ${defaultParams.termSmoothing}") + .text(s"amount of term (word) smoothing to use (-1=auto). default: ${defaultParams.termSmoothing}") .action((x, c) => c.copy(termSmoothing = x)) opt[Int]("vocabSize") .text(s"number of distinct word types to use, chosen by frequency." + @@ -135,7 +135,7 @@ object LDAExample { } /** - * Load documents, tokenize them, create vocabulary, and prepare documents as word count vectors. + * Load documents, tokenize them, create vocabulary, and prepare documents as term count vectors. */ private def preprocess( sc: SparkContext, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index b4a80a16fa76f..c405a6ca7d93e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -93,7 +93,13 @@ class LDA private ( * - The 50/k is common in LDA libraries. * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. */ - def getTopicSmoothing: Double = topicSmoothing + def getTopicSmoothing: Double = { + if (this.topicSmoothing > 0) { + this.topicSmoothing + } else { + (50.0 / k) + 1.0 + } + } def setTopicSmoothing(topicSmoothing: Double): this.type = { require(topicSmoothing > 0.0 || topicSmoothing == -1.0, @@ -108,7 +114,7 @@ class LDA private ( /** * Term smoothing parameter (commonly named "eta"). * - * This is the parameter to the Dirichlet prior placed on the per-topic word distributions + * This is the parameter to the Dirichlet prior placed on the per-topic term distributions * (which 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.) * @@ -121,7 +127,13 @@ class LDA private ( * - The 0.1 gives a small amount of smoothing. * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. */ - def getTermSmoothing: Double = termSmoothing + def getTermSmoothing: Double = { + if (this.termSmoothing > 0) { + this.termSmoothing + } else { + 1.1 + } + } def setTermSmoothing(termSmoothing: Double): this.type = { require(termSmoothing > 0.0 || termSmoothing == -1.0, @@ -160,17 +172,7 @@ class LDA private ( * @return Inferred LDA model */ def run(documents: RDD[Document]): DistributedLDAModel = { - val topicSmoothing = if (this.topicSmoothing > 0) { - this.topicSmoothing - } else { - (50.0 / k) + 1.0 - } - val termSmoothing = if (this.termSmoothing > 0) { - this.termSmoothing - } else { - 1.1 - } - var state = LDA.initialState(documents, k, topicSmoothing, termSmoothing, seed) + var state = LDA.initialState(documents, k, getTopicSmoothing, getTermSmoothing, seed) var iter = 0 while (iter < maxIterations) { state = state.next() @@ -390,14 +392,14 @@ object LDA { */ private def computePTopic( docTopicCounts: TopicCounts, - wordTopicCounts: TopicCounts, + termTopicCounts: TopicCounts, totalTopicCounts: TopicCounts, vocabSize: Int, eta: Double, alpha: Double): TopicCounts = { val K = docTopicCounts.length val N_j = docTopicCounts.data - val N_w = wordTopicCounts.data + val N_w = termTopicCounts.data val N = totalTopicCounts.data val eta1 = eta - 1.0 val alpha1 = alpha - 1.0 From 0b9039317aa26be722d98e19437f5a9541e0ccb2 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sat, 24 Jan 2015 18:41:00 -0800 Subject: [PATCH 17/30] renamed LDA LearningState.collectTopicTotals to globalTopicTotals --- .../scala/org/apache/spark/mllib/clustering/LDA.scala | 8 ++++---- .../org/apache/spark/mllib/clustering/LDAModel.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index c405a6ca7d93e..d15d28643510c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -289,7 +289,7 @@ object LDA { val W = vocabSize val alpha = topicSmoothing - val N_k = collectTopicTotals() + val N_k = globalTopicTotals() val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = (edgeContext) => { // Compute N_{wj} gamma_{wjk} @@ -323,7 +323,7 @@ object LDA { graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get } } - def collectTopicTotals(): TopicCounts = { + def globalTopicTotals(): TopicCounts = { val numTopics = k graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) } @@ -342,7 +342,7 @@ object LDA { val alpha = topicSmoothing assert(eta > 1.0) assert(alpha > 1.0) - val N_k = collectTopicTotals() + val N_k = globalTopicTotals() val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) // Edges: Compute token log probability from phi_{wk}, theta_{kj}. val sendMsg: EdgeContext[TopicCounts, TokenCount, Double] => Unit = (edgeContext) => { @@ -367,7 +367,7 @@ object LDA { val alpha = topicSmoothing // Term vertices: Compute phi_{wk}. Use to compute prior log probability. // Doc vertex: Compute theta_{kj}. Use to compute prior log probability. - val N_k = collectTopicTotals() + val N_k = globalTopicTotals() val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) val seqOp: (Double, (VertexId, TopicCounts)) => Double = { case (sumPrior: Double, vertex: (VertexId, TopicCounts)) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 66b3ded6fa5d8..749c5661c14ad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -202,7 +202,7 @@ class DistributedLDAModel private[clustering] ( val numTopics = k // Note: N_k is not needed to find the top terms, but it is needed to normalize weights // to a distribution over terms. - val N_k: TopicCounts = state.collectTopicTotals() + val N_k: TopicCounts = state.globalTopicTotals() val topicSummary = state.graph.vertices.filter(isTermVertex) .mapPartitions { termVertices => // For this partition, collect the most common terms for each topic in queues: From 43c1c403723ae95dbe14fdbafbbffb560bdf1d86 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 26 Jan 2015 13:12:57 -0800 Subject: [PATCH 18/30] small cleanup --- .../main/scala/org/apache/spark/mllib/clustering/LDA.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index d15d28643510c..7742c0a50c3ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -228,6 +228,10 @@ object LDA { - Term update: - N_{wk} <- sum_j N_{wj} gamma_{wjk} - N_k <- sum_w N_{wk} + + TODO: Add simplex constraints to allow alpha in (0,1). + See: Vorontsov and Potapenko. "Tutorial on Probabilistic Topic Modeling : Additive + Regularization for Stochastic Matrix Factorization." 2014. */ /** @@ -419,7 +423,6 @@ object LDA { /** * Compute bipartite term/doc graph. - * doc ids are shifted by vocabSize to maintain uniqueness */ private def initialState( docs: RDD[Document], From cb5a31923bf7db1bf24a10939ccd6b3f19bb0f13 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 29 Jan 2015 11:05:26 -0800 Subject: [PATCH 19/30] Added checkpointing to LDA * new class PeriodicGraphCheckpointer * params checkpointDir, checkpointInterval to LDA Internal changes to LDA * Changed State to be mutable (since it needs to hold the PeriodicGraphCheckpointer) * Added timing instrumentation * Changed DistributedLDAModel not to hold a LearningState * This was needed since LearningState needs to hold a PeriodicGraphCheckpointer. We should be able to copy a model, but we cannot copy PeriodicGraphCheckpointer instances. Public changes to LDA * Updated naming of describeTopics, and commented out version using String --- .../apache/spark/mllib/clustering/LDA.scala | 164 ++++++++++-------- .../spark/mllib/clustering/LDAModel.scala | 136 ++++++++++++--- .../impl/PeriodicGraphCheckpointer.scala | 150 ++++++++++++++++ 3 files changed, 347 insertions(+), 103 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 7742c0a50c3ca..29b7593df9dc8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -24,6 +24,8 @@ import breeze.linalg.{DenseVector => BDV, sum => brzSum, normalize, axpy => brzA import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.GraphImpl +import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -59,12 +61,14 @@ class LDA private ( private var maxIterations: Int, private var topicSmoothing: Double, private var termSmoothing: Double, - private var seed: Long) extends Logging { + private var seed: Long, + private var checkpointDir: Option[String], + private var checkpointInterval: Int) extends Logging { import LDA._ def this() = this(k = 10, maxIterations = 20, topicSmoothing = -1, termSmoothing = -1, - seed = Utils.random.nextLong()) + seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) /** * Number of topics to infer. I.e., the number of soft cluster centers. @@ -164,6 +168,35 @@ class LDA private ( this } + /** + * Directory for storing checkpoint files during learning. + * This is not necessary, but checkpointing helps with recovery (when nodes fail). + * It also helps with eliminating temporary shuffle files on disk, which can be important when + * LDA is run for many iterations. + */ + def getCheckpointDir: Option[String] = checkpointDir + + def setCheckpointDir(checkpointDir: String): this.type = { + this.checkpointDir = Some(checkpointDir) + this + } + + def clearCheckpointDir(): this.type = { + this.checkpointDir = None + this + } + + /** + * Period (in iterations) between checkpoints. + * @see [[getCheckpointDir]] + */ + def getCheckpointInterval: Int = checkpointInterval + + def setCheckpointInterval(checkpointInterval: Int): this.type = { + this.checkpointInterval = checkpointInterval + this + } + /** * Learn an LDA model using the given dataset. * @@ -172,13 +205,19 @@ class LDA private ( * @return Inferred LDA model */ def run(documents: RDD[Document]): DistributedLDAModel = { - var state = LDA.initialState(documents, k, getTopicSmoothing, getTermSmoothing, seed) + var state = LDA.initialState(documents, k, getTopicSmoothing, getTermSmoothing, seed, + checkpointDir, checkpointInterval) var iter = 0 + val iterationTimes = Array.fill[Double](maxIterations)(0) while (iter < maxIterations) { - state = state.next() + val start = System.nanoTime() + state.next() + val elapsedSeconds = (System.nanoTime() - start) / 1e9 + iterationTimes(iter) = elapsedSeconds iter += 1 } - new DistributedLDAModel(state) + state.graphCheckpointer.deleteAllCheckpoints() + new DistributedLDAModel(state, iterationTimes) } } @@ -279,21 +318,24 @@ object LDA { * @param topicSmoothing "alpha" * @param termSmoothing "eta" */ - private[clustering] case class LearningState( - graph: Graph[TopicCounts, TokenCount], - k: Int, - vocabSize: Int, - topicSmoothing: Double, - termSmoothing: Double) { - // TODO: Checkpoint periodically? - def next(): LearningState = copy(graph = step(graph)) - - private def step(graph: Graph[TopicCounts, TokenCount]): Graph[TopicCounts, TokenCount] = { + private[clustering] class LearningState( + var graph: Graph[TopicCounts, TokenCount], + val k: Int, + val vocabSize: Int, + val topicSmoothing: Double, + val termSmoothing: Double, + checkpointDir: Option[String], + checkpointInterval: Int) { + + private[LDA] val graphCheckpointer = new PeriodicGraphCheckpointer[TopicCounts, TokenCount]( + graph, checkpointDir, checkpointInterval) + + def next(): LearningState = { val eta = termSmoothing val W = vocabSize val alpha = topicSmoothing - val N_k = globalTopicTotals() + val N_k = globalTopicTotals val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = (edgeContext) => { // Compute N_{wj} gamma_{wjk} @@ -306,7 +348,7 @@ object LDA { edgeContext.sendToSrc((false, scaledTopicDistribution)) } // This is a hack to detect whether we could modify the values in-place. - // TODO: Add zero/seqOp/combOp option to aggregateMessages. + // TODO: Add zero/seqOp/combOp option to aggregateMessages. (SPARK-5438) val mergeMsg: ((Boolean, TopicCounts), (Boolean, TopicCounts)) => (Boolean, TopicCounts) = (m0, m1) => { val sum = @@ -324,71 +366,35 @@ object LDA { graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) .mapValues(_._2) // Update the vertex descriptors with the new counts. - graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get } - } - - def globalTopicTotals(): TopicCounts = { - val numTopics = k - graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) + val newGraph = + GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) + // graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get } + /* + previousGraph match { + case Some(prevG) => + prevG.unpersist(blocking = false) + case None => + } + copy(graph = newGraph, previousGraph = Some(graph)) + */ + graph = newGraph + graphCheckpointer.updateGraph(newGraph) + globalTopicTotals = computeGlobalTopicTotals() + this } /** - * Compute the log likelihood of the observed tokens, given the current parameter estimates: - * log P(docs | topics, topic distributions for docs, alpha, eta) + * Aggregate distributions over topics from all term vertices. * - * Note: - * - This excludes the prior; for that, use [[logPrior]]. - * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the - * hyperparameters. + * Note: This executes an action on the graph RDDs. */ - lazy val logLikelihood: Double = { - val eta = termSmoothing - val alpha = topicSmoothing - assert(eta > 1.0) - assert(alpha > 1.0) - val N_k = globalTopicTotals() - val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) - // Edges: Compute token log probability from phi_{wk}, theta_{kj}. - val sendMsg: EdgeContext[TopicCounts, TokenCount, Double] => Unit = (edgeContext) => { - val N_wj = edgeContext.attr - val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) - val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) - val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k - val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) - val tokenLogLikelihood = N_wj * math.log(phi_wk.dot(theta_kj)) - edgeContext.sendToDst(tokenLogLikelihood) - } - graph.aggregateMessages[Double](sendMsg, _ + _) - .map(_._2).fold(0.0)(_ + _) - } + private[clustering] var globalTopicTotals: TopicCounts = computeGlobalTopicTotals() - /** - * Compute the log probability of the current parameter estimate: - * log P(topics, topic distributions for docs | alpha, eta) - */ - lazy val logPrior: Double = { - val eta = termSmoothing - val alpha = topicSmoothing - // Term vertices: Compute phi_{wk}. Use to compute prior log probability. - // Doc vertex: Compute theta_{kj}. Use to compute prior log probability. - val N_k = globalTopicTotals() - val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) - val seqOp: (Double, (VertexId, TopicCounts)) => Double = { - case (sumPrior: Double, vertex: (VertexId, TopicCounts)) => - if (isTermVertex(vertex)) { - val N_wk = vertex._2 - val smoothed_N_wk: TopicCounts = N_wk + (eta - 1.0) - val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k - (eta - 1.0) * brzSum(phi_wk.map(math.log)) - } else { - val N_kj = vertex._2 - val smoothed_N_kj: TopicCounts = N_kj + (alpha - 1.0) - val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) - (alpha - 1.0) * brzSum(theta_kj.map(math.log)) - } - } - graph.vertices.aggregate(0.0)(seqOp, _ + _) + private def computeGlobalTopicTotals(): TopicCounts = { + val numTopics = k + graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) } + } /** @@ -429,7 +435,9 @@ object LDA { k: Int, topicSmoothing: Double, termSmoothing: Double, - randomSeed: Long): LearningState = { + randomSeed: Long, + checkpointDir: Option[String], + checkpointInterval: Int): LearningState = { // For each document, create an edge (Document -> Term) for each unique term in the document. val edges: RDD[Edge[TokenCount]] = docs.flatMap { doc => // Add edges for terms with non-zero counts. @@ -472,6 +480,8 @@ object LDA { val graph = Graph(docVertices ++ termVertices, edges) .partitionBy(PartitionStrategy.EdgePartition1D) - LearningState(graph, k, vocabSize, topicSmoothing, termSmoothing) + new LearningState(graph, k, vocabSize, topicSmoothing, termSmoothing, checkpointDir, + checkpointInterval) } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 749c5661c14ad..279e6ed99c71e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.clustering -import breeze.linalg.{DenseMatrix => BDM, normalize} +import breeze.linalg.{DenseMatrix => BDM, normalize, sum => brzSum} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.graphx.{VertexId, EdgeContext, Graph} import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} import org.apache.spark.rdd.RDD import org.apache.spark.util.BoundedPriorityQueue @@ -62,7 +63,7 @@ abstract class LDAModel private[clustering] { * as (term weight in topic, term index). * Each topic's terms are sorted in order of decreasing weight. */ - def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] + def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] /** * Return the topics described by weighted terms. @@ -73,7 +74,43 @@ abstract class LDAModel private[clustering] { * as (term weight in topic, term index). * Each topic's terms are sorted in order of decreasing weight. */ - def describeTopics(): Array[Array[(Double, String)]] = describeTopics(vocabSize) + def describeTopics(): Array[Array[(Double, Int)]] = describeTopics(vocabSize) + + /* TODO (once LDA can be trained with Strings or given a dictionary) + * Return the topics described by weighted terms. + * + * This is similar to [[describeTopics()]] but returns String values for terms. + * If this model was trained using Strings or was given a dictionary, then this method returns + * terms as text. Otherwise, this method returns terms as term indices. + * + * This limits the number of terms per topic. + * This is approximate; it may not return exactly the top-weighted terms for each topic. + * To get a more precise set of top terms, increase maxTermsPerTopic. + * + * @param maxTermsPerTopic Maximum number of terms to collect for each topic. + * @return Array over topics, where each element is a set of top terms represented + * as (term weight in topic, term), where "term" is either the actual term text + * (if available) or the term index. + * Each topic's terms are sorted in order of decreasing weight. + */ + //def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[Array[(Double, String)]] + + /* TODO (once LDA can be trained with Strings or given a dictionary) + * Return the topics described by weighted terms. + * + * This is similar to [[describeTopics()]] but returns String values for terms. + * If this model was trained using Strings or was given a dictionary, then this method returns + * terms as text. Otherwise, this method returns terms as term indices. + * + * WARNING: If vocabSize and k are large, this can return a large object! + * + * @return Array over topics, where each element is a set of top terms represented + * as (term weight in topic, term), where "term" is either the actual term text + * (if available) or the term index. + * Each topic's terms are sorted in order of decreasing weight. + */ + //def describeTopicsAsStrings(): Array[Array[(Double, String)]] = + // describeTopicsAsStrings(vocabSize) /* TODO * Compute the log likelihood of the observed tokens, given the current parameter estimates: @@ -131,15 +168,12 @@ class LocalLDAModel private[clustering] ( override def topicsMatrix: Matrix = topics - override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] = { + override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { val brzTopics = topics.toBreeze.toDenseMatrix - val topicSummary = Range(0, k).map { topicIndex => + Range(0, k).map { topicIndex => val topic = normalize(brzTopics(::, topicIndex), 1.0) topic.toArray.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic) }.toArray - topicSummary.map { topic => - topic.map { case (weight, term) => (weight, term.toString) } - } } // TODO @@ -162,16 +196,28 @@ class LocalLDAModel private[clustering] ( * API changes. */ @DeveloperApi -class DistributedLDAModel private[clustering] ( - private val state: LDA.LearningState) extends LDAModel { +class DistributedLDAModel private ( + private val graph: Graph[LDA.TopicCounts, LDA.TokenCount], + private val globalTopicTotals: LDA.TopicCounts, + val k: Int, + val vocabSize: Int, + private val topicSmoothing: Double, + private val termSmoothing: Double, + private[spark] val iterationTimes: Array[Double]) extends LDAModel { import LDA._ - def toLocal: LocalLDAModel = new LocalLDAModel(topicsMatrix) - - override def k: Int = state.k + private[clustering] def this(state: LDA.LearningState, iterationTimes: Array[Double]) = { + this(state.graph, state.globalTopicTotals, state.k, state.vocabSize, state.topicSmoothing, + state.termSmoothing, iterationTimes) + } - override def vocabSize: Int = state.vocabSize + /** + * Convert model to a local model. + * The local model stores the inferred topics but not the topic distributions for training + * documents. + */ + def toLocal: LocalLDAModel = new LocalLDAModel(topicsMatrix) /** * Inferred topics, where each topic is represented by a distribution over terms. @@ -183,7 +229,7 @@ class DistributedLDAModel private[clustering] ( override lazy val topicsMatrix: Matrix = { // Collect row-major topics val termTopicCounts: Array[(Int, TopicCounts)] = - state.graph.vertices.filter(_._1 < 0).map { case (termIndex, cnts) => + graph.vertices.filter(_._1 < 0).map { case (termIndex, cnts) => (index2term(termIndex), cnts) }.collect() // Convert to Matrix @@ -198,12 +244,12 @@ class DistributedLDAModel private[clustering] ( Matrices.fromBreeze(brzTopics) } - override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, String)]] = { + override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { val numTopics = k // Note: N_k is not needed to find the top terms, but it is needed to normalize weights // to a distribution over terms. - val N_k: TopicCounts = state.globalTopicTotals() - val topicSummary = state.graph.vertices.filter(isTermVertex) + val N_k: TopicCounts = globalTopicTotals + graph.vertices.filter(isTermVertex) .mapPartitions { termVertices => // For this partition, collect the most common terms for each topic in queues: // queues(topic) = queue of (term weight, term index). @@ -221,16 +267,13 @@ class DistributedLDAModel private[clustering] ( q1.zip(q2).foreach { case (a, b) => a ++= b} q1 }.map(_.toArray.sortBy(-_._1)) - topicSummary.map { topic => - topic.map { case (weight, term) => (weight, term.toString) } - } } // TODO // override def logLikelihood(documents: RDD[Document]): Double = ??? /** - * Compute the log likelihood of the observed tokens in the training set, + * Log likelihood of the observed tokens in the training set, * given the current parameter estimates: * log P(docs | topics, topic distributions for docs, alpha, eta) * @@ -239,13 +282,54 @@ class DistributedLDAModel private[clustering] ( * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the * hyperparameters. */ - def logLikelihood = state.logLikelihood + lazy val logLikelihood: Double = { + val eta = termSmoothing + val alpha = topicSmoothing + assert(eta > 1.0) + assert(alpha > 1.0) + val N_k = globalTopicTotals + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + // Edges: Compute token log probability from phi_{wk}, theta_{kj}. + val sendMsg: EdgeContext[TopicCounts, TokenCount, Double] => Unit = (edgeContext) => { + val N_wj = edgeContext.attr + val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) + val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) + val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) + val tokenLogLikelihood = N_wj * math.log(phi_wk.dot(theta_kj)) + edgeContext.sendToDst(tokenLogLikelihood) + } + graph.aggregateMessages[Double](sendMsg, _ + _) + .map(_._2).fold(0.0)(_ + _) + } /** - * Compute the log probability of the current parameter estimate, under the prior: + * Log probability of the current parameter estimate: * log P(topics, topic distributions for docs | alpha, eta) */ - def logPrior = state.logPrior + lazy val logPrior: Double = { + val eta = termSmoothing + val alpha = topicSmoothing + // Term vertices: Compute phi_{wk}. Use to compute prior log probability. + // Doc vertex: Compute theta_{kj}. Use to compute prior log probability. + val N_k = globalTopicTotals + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + val seqOp: (Double, (VertexId, TopicCounts)) => Double = { + case (sumPrior: Double, vertex: (VertexId, TopicCounts)) => + if (isTermVertex(vertex)) { + val N_wk = vertex._2 + val smoothed_N_wk: TopicCounts = N_wk + (eta - 1.0) + val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + (eta - 1.0) * brzSum(phi_wk.map(math.log)) + } else { + val N_kj = vertex._2 + val smoothed_N_kj: TopicCounts = N_kj + (alpha - 1.0) + val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) + (alpha - 1.0) * brzSum(theta_kj.map(math.log)) + } + } + graph.vertices.aggregate(0.0)(seqOp, _ + _) + } /** * For each document in the training set, return the distribution over topics for that document @@ -254,7 +338,7 @@ class DistributedLDAModel private[clustering] ( * @return RDD of (document ID, topic distribution) pairs */ def topicDistributions: RDD[(Long, Vector)] = { - state.graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => + graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => (docID.toLong, Vectors.fromBreeze(normalize(topicCounts, 1.0))) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala new file mode 100644 index 0000000000000..56259cddd5e51 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -0,0 +1,150 @@ +/* + * 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.mllib.impl + +import scala.collection.mutable + +import org.apache.hadoop.fs.{Path, FileSystem} + +import org.apache.spark.Logging +import org.apache.spark.graphx.Graph +import org.apache.spark.storage.StorageLevel + + +/** + * This class helps with persisting and checkpointing Graphs. + * + * This class maintains a FIFO queue of Graphs, each of which is persisted and some of which are + * checkpointed. Once one Graph has been checkpointed, then previous RDDs are unpersisted and their + * checkpoint files are removed. + * + * Users should call [[PeriodicGraphCheckpointer.updateGraph()]] when a new graph has been created, + * before the graph has been materialized. + * + * NOTE: This class should NOT be copied (since copies may conflict on which Graphs should be + * checkpointed). + * + * @param currentGraph Initial graph + * @param checkpointDir The directory for storing checkpoint files + * @param checkpointInterval Graphs will be checkpointed at this interval + * @tparam VD Vertex descriptor type + * @tparam ED Edge descriptor type + */ +private[mllib] class PeriodicGraphCheckpointer[VD, ED]( + var currentGraph: Graph[VD, ED], + val checkpointDir: Option[String], + val checkpointInterval: Int) extends Logging { + + /** FIFO queue of past checkpointed RDDs*/ + private val checkpointQueue = mutable.Queue[Graph[VD, ED]]() + + /** FIFO queue of past persisted RDDs*/ + private val persistedQueue = mutable.Queue[Graph[VD, ED]]() + + /** Number of times [[updateGraph()]] has been called */ + private var updateCount = 0 + + /** + * Spark Context for the Graphs given to this checkpointer. + * NOTE: This code assumes that only one SparkContext is used for the given graphs. + */ + private val sc = currentGraph.vertices.sparkContext + + // If a checkpoint directory is given, and there's no prior checkpoint directory, + // then set the checkpoint directory with the given one. + if (checkpointDir.nonEmpty && sc.getCheckpointDir.isEmpty) { + sc.setCheckpointDir(checkpointDir.get) + } + + updateGraph(currentGraph) + + /** + * Update [[currentGraph]] with a new graph. Handle persistence and checkpointing as needed. + * Since this handles persistence and checkpointing, this should be called before the graph + * has been materialized. + * + * @param newGraph New graph created from previous graphs in the lineage. + */ + def updateGraph(newGraph: Graph[VD, ED]): Unit = { + if (newGraph.vertices.getStorageLevel == StorageLevel.NONE) { + println(s"PeriodicGraphCheckpointer.updateGraph: persisting ${newGraph.vertices.id}") + newGraph.persist() + } + persistedQueue.enqueue(newGraph) + // We try to maintain 2 Graphs in persistedQueue to support the semantics of this class: + // Users should call [[updateGraph()]] when a new graph has been created, + // before the graph has been materialized. + while (persistedQueue.size > 3) { + val graphToUnpersist = persistedQueue.dequeue() + println(s"PeriodicGraphCheckpointer.updateGraph: unpersisting ${graphToUnpersist.vertices.id}") + graphToUnpersist.unpersist(blocking = false) + } + updateCount += 1 + + // Handle checkpointing (after persisting) + if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) { + // Add new checkpoint before removing old checkpoints. + println(s"PeriodicGraphCheckpointer.updateGraph: checkpointing ${newGraph.vertices.id}") + newGraph.checkpoint() + checkpointQueue.enqueue(newGraph) + // Remove checkpoints before the latest one. + var canDelete = true + while (checkpointQueue.size > 1 && canDelete) { + // Delete the oldest checkpoint only if the next checkpoint exists. + if (checkpointQueue.get(1).get.isCheckpointed) { + removeCheckpointFile() + } else { + canDelete = false + } + } + } + } + + /** + * Call this at the end to delete any remaining checkpoint files. + */ + def deleteAllCheckpoints(): Unit = { + while (checkpointQueue.size > 0) { + removeCheckpointFile() + } + } + + /** + * Dequeue the oldest checkpointed Graph, and remove its checkpoint files. + * This prints a warning but does not fail if the files cannot be removed. + */ + private def removeCheckpointFile(): Unit = { + val old = checkpointQueue.dequeue() + println(s"PeriodicGraphCheckpointer.updateGraph: removing checkpoint ${old.vertices.id}") + // Since the old checkpoint is not deleted by Spark, we manually delete it. + val fs = FileSystem.get(sc.hadoopConfiguration) + old.getCheckpointFiles.foreach { checkpointFile => + try { + println(s" --removing file: $checkpointFile") + fs.delete(new Path(checkpointFile), true) + } catch { + case e: Exception => + println("PeriodicGraphCheckpointer could not remove old checkpoint file: " + + checkpointFile) + logWarning("PeriodicGraphCheckpointer could not remove old checkpoint file: " + + checkpointFile) + } + } + } + +} From 993ca566179fe7567632357a3b24b85f35f843db Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 29 Jan 2015 16:31:27 -0800 Subject: [PATCH 20/30] =?UTF-8?q?*=20Removed=20Document=20type=20in=20favo?= =?UTF-8?q?r=20of=20(Long,=20Vector)=20*=20Changed=20doc=20ID=20restrictio?= =?UTF-8?q?n=20to=20be:=20id=20must=20be=20nonnegative=20and=20unique=20in?= =?UTF-8?q?=20the=20doc=20(instead=20of=200,1,2,...)=20*=20Add=20checks=20?= =?UTF-8?q?for=20valid=20ranges=20of=20eta,=20alpha=20*=20Rename=20?= =?UTF-8?q?=E2=80=9CLearningState=E2=80=9D=20to=20=E2=80=9CEMOptimizer?= =?UTF-8?q?=E2=80=9D=20*=20Renamed=20params:=20termSmoothing=20->=20topicC?= =?UTF-8?q?oncentration,=20topicSmoothing=20->=20docConcentration=20=20=20?= =?UTF-8?q?*=20Also=20added=20aliases=20alpha,=20beta?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Also added LDATiming, to be removed shortly --- .../spark/examples/mllib/LDAExample.scala | 32 +- .../spark/examples/mllib/LDATiming.scala | 339 ++++++++++++++++++ .../apache/spark/mllib/clustering/LDA.scala | 154 ++++---- .../spark/mllib/clustering/LDAModel.scala | 42 ++- .../spark/mllib/clustering/LDASuite.scala | 17 +- 5 files changed, 457 insertions(+), 127 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 718bbaffab73f..8053761a802db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -26,10 +26,8 @@ import scopt.OptionParser import org.apache.log4j.{Level, Logger} import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.SparkContext._ import org.apache.spark.mllib.clustering.LDA -import org.apache.spark.mllib.clustering.LDA.Document -import org.apache.spark.mllib.linalg.SparseVector +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD @@ -46,8 +44,8 @@ object LDAExample { input: Seq[String] = Seq.empty, k: Int = 20, maxIterations: Int = 10, - topicSmoothing: Double = -1, - termSmoothing: Double = -1, + docConcentration: Double = -1, + topicConcentration: Double = -1, vocabSize: Int = 10000, stopwordFile: String = "") extends AbstractParams[Params] @@ -62,12 +60,14 @@ object LDAExample { opt[Int]("maxIterations") .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) - opt[Double]("topicSmoothing") - .text(s"amount of topic smoothing to use (-1=auto). default: ${defaultParams.topicSmoothing}") - .action((x, c) => c.copy(topicSmoothing = x)) - opt[Double]("termSmoothing") - .text(s"amount of term (word) smoothing to use (-1=auto). default: ${defaultParams.termSmoothing}") - .action((x, c) => c.copy(termSmoothing = x)) + opt[Double]("docConcentration") + .text(s"amount of topic smoothing to use (> 1.0) (-1=auto)." + + s" default: ${defaultParams.docConcentration}") + .action((x, c) => c.copy(docConcentration = x)) + opt[Double]("topicConcentration") + .text(s"amount of term (word) smoothing to use (> 1.0) (-1=auto)." + + s" default: ${defaultParams.topicConcentration}") + .action((x, c) => c.copy(topicConcentration = x)) opt[Int]("vocabSize") .text(s"number of distinct word types to use, chosen by frequency." + s" default: ${defaultParams.vocabSize}") @@ -105,8 +105,8 @@ object LDAExample { val lda = new LDA() lda.setK(params.k) .setMaxIterations(params.maxIterations) - .setTopicSmoothing(params.topicSmoothing) - .setTermSmoothing(params.termSmoothing) + .setDocConcentration(params.docConcentration) + .setTopicConcentration(params.topicConcentration) val startTime = System.nanoTime() val ldaModel = lda.run(corpus) val elapsed = (System.nanoTime() - startTime) / 1e9 @@ -141,7 +141,7 @@ object LDAExample { sc: SparkContext, paths: Seq[String], vocabSize: Int, - stopwordFile: String): (RDD[Document], Array[String]) = { + stopwordFile: String): (RDD[(Long, Vector)], Array[String]) = { val files: Seq[RDD[(String, String)]] = for (p <- paths) yield { sc.wholeTextFiles(p) @@ -183,8 +183,8 @@ object LDAExample { val indices = wc.keys.toArray.sorted val values = indices.map(i => wc(i).toDouble) - val sb = new SparseVector(vocab.size, indices, values) - LDA.Document(sb, id) + val sb = Vectors.sparse(vocab.size, indices, values) + (id, sb) } val vocabArray = new Array[String](vocab.size) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala new file mode 100644 index 0000000000000..406dc31b392c3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala @@ -0,0 +1,339 @@ +/* + * 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.examples.mllib + +import scala.collection.mutable + +import java.text.BreakIterator + +import scopt.OptionParser + +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.mllib.clustering.LDA +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.collection.OpenHashSet + +/** + * An app for timing Latent Dirichlet Allocation (LDA). + * + * This takes vectors of parameters: + * - corpusSize + * - vocabSize + * - k + * For each combination of values, it runs LDA and prints the time for iterations, + * the topics estimated, etc. + * + * Run with + * {{{ + * ./bin/run-example mllib.LDATiming [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object LDATiming { + + private case class Params( + input: Seq[String] = Seq.empty, + corpusSizes: Array[Long] = Array(-1), + ks: Array[Int] = Array(20), + numPartitions: Int = 16, + maxIterations: Int = 10, + docConcentration: Double = 0.1, + topicConcentration: Double = 0.1, + vocabSizes: Array[Int] = Array(10000), + stopwordFile: String = "", + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("LDATiming") { + head("LDATiming: an example LDA timing app for plain text data.") + opt[String]("corpusSizes") + .text(s"numbers of documents to test. default: ${defaultParams.corpusSizes}") + .action((x, c) => c.copy(corpusSizes = x.split("\\s").map(_.toLong))) + opt[String]("ks") + .text(s"numbers of topics to test. default: ${defaultParams.ks}") + .action((x, c) => c.copy(ks = x.split("\\s").map(_.toInt))) + opt[Int]("maxIterations") + .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}") + .action((x, c) => c.copy(maxIterations = x)) + opt[Int]("numPartitions") + .text(s"number of partitions. default: ${defaultParams.numPartitions}") + .action((x, c) => c.copy(numPartitions = x)) + opt[Double]("docConcentration") + .text(s"amount of topic smoothing to use. default: ${defaultParams.docConcentration}") + .action((x, c) => c.copy(docConcentration = x)) + opt[Double]("topicConcentration") + .text(s"amount of word smoothing to use. default: ${defaultParams.topicConcentration}") + .action((x, c) => c.copy(topicConcentration = x)) + opt[String]("vocabSizes") + .text(s"numbers of distinct word types to use, chosen by frequency." + + s" default: ${defaultParams.vocabSizes}") + .action((x, c) => c.copy(vocabSizes = x.split("\\s").map(_.toInt))) + opt[String]("stopwordFile") + .text(s"filepath for a list of stopwords. Note: This must fit on a single machine." + + s" default: ${defaultParams.stopwordFile}") + .action((x, c) => c.copy(stopwordFile = x)) + opt[String]("checkpointDir") + .text(s"Directory for checkpointing intermediate results." + + s" Checkpointing helps with recovery and eliminates temporary shuffle files on disk." + + s" default: ${defaultParams.checkpointDir}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"Iterations between each checkpoint. Only used if checkpointDir is set." + + s" default: ${defaultParams.checkpointInterval}") + .action((x, c) => c.copy(checkpointInterval = x)) + arg[String]("...") + .text("input paths (directories) to plain text corpora. Each text file line should hold 1 document.") + .unbounded() + .required() + .action((x, c) => c.copy(input = c.input :+ x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + parser.showUsageAsError + sys.exit(1) + } + } + + private def run(params: Params) { + val conf = new SparkConf().setAppName(s"LDATiming with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + // Load documents, and prepare them for LDA. + for (corpusSize <- params.corpusSizes) { + for (vocabSize <- params.vocabSizes) { + + val preprocessStart = System.nanoTime() + val (corpus, vocabArray) = preprocess(sc, params.input, corpusSize, vocabSize, params.stopwordFile) + corpus.repartition(params.numPartitions).cache() // cache since LDA is iterative + val actualCorpusSize = corpus.count() + val actualVocabSize = vocabArray.size + val actualNumTokens = corpus.map(_._2.toArray.sum.toLong).sum().toLong + val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 + + println() + println(s"DATASET with corpusSize=$corpusSize, vocabSize=$vocabSize:") + println(s"\t Training set size: $actualCorpusSize documents") + println(s"\t Vocabulary size: $actualVocabSize terms") + println(s"\t Training set size: $actualNumTokens tokens") + println(s"\t Preprocessing time: $preprocessElapsed sec") + + for (k <- params.ks) { + // Run LDA. + val lda = new LDA() + lda.setK(k) + .setMaxIterations(params.maxIterations) + .setDocConcentration(params.docConcentration) + .setTopicConcentration(params.topicConcentration) + .setCheckpointInterval(params.checkpointInterval) + if (params.checkpointDir.nonEmpty) { + lda.setCheckpointDir(params.checkpointDir.get) + } + val startTime = System.nanoTime() + val ldaModel = lda.run(corpus) + val elapsed = (System.nanoTime() - startTime) / 1e9 + + println(s"Finished training LDA model. Summary:") + println(s"\t Training time: $elapsed sec") + val avgLogLikelihood = ldaModel.logLikelihood / actualCorpusSize.toDouble + println(s"\t Training data average log likelihood: $avgLogLikelihood") + println(s"\t Training times per iteration (sec):\n${ldaModel.iterationTimes.mkString("\t", "\n\t", "\n")}") + println() + + // Print the topics, showing the top-weighted terms for each topic. + val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) + val topics = topicIndices.map { topic => + topic.map { case (weight, term) => (weight, vocabArray(term.toInt))} + } + println(s"$k topics:") + topics.zipWithIndex.foreach { case (topic, i) => + println(s"TOPIC $i") + topic.foreach { case (weight, term) => + println(s"$term\t$weight") + } + println() + } + println() + println("--------------------------------------------------------------") + println() + println(s"RESULTS: $corpusSize $vocabSize $actualCorpusSize $actualVocabSize $actualNumTokens $k $elapsed $avgLogLikelihood ${ldaModel.iterationTimes.mkString(" ")}") + println() + println("==============================================================") + println() + } + } + } + + sc.stop() + } + + /** + * Load documents, tokenize them, create vocabulary, and prepare documents as word count vectors. + */ + private def preprocess( + sc: SparkContext, + paths: Seq[String], + corpusSize: Long, + vocabSize: Int, + stopwordFile: String): (RDD[(Long, Vector)], Array[String]) = { + + // Get dataset of document texts + // One document per line in each text file. + val files: Seq[RDD[String]] = for (p <- paths) yield { + sc.textFile(p) + } + val textRDD_tmp: RDD[String] = files.reduce(_ ++ _) // combine results from multiple paths + val origSize = textRDD_tmp.count() + + // Subsample data. + val textRDD: RDD[String] = if (corpusSize == -1 || corpusSize >= origSize) { + textRDD_tmp + } else { + textRDD_tmp.sample(withReplacement = true, fraction = corpusSize.toDouble / origSize, + seed = 123456) + } + + // Split text into words + val tokenizer = new Tokenizer(sc, stopwordFile) + val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => + id -> tokenizer.getWords(text) + } + + /* + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Int)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1) } + .reduceByKey(_ + _) + */ + + // Choose vocabulary: Map[word -> id] + val vocab: Map[String, Int] = if (vocabSize == -1) { + val allWords = tokenized.aggregate(new OpenHashSet[String])({ + case (wordSet, (docId, words)) => + words.foreach(word => wordSet.add(word)) + wordSet + }, { case (a, b) => + b.iterator.foreach(w => a.add(w)) + a + }) + allWords + .iterator + .zipWithIndex + .toMap + } else { + val allWords = tokenized.aggregate(new mutable.HashMap[String, Long])({ + case (wc, (docId, words)) => + words.foreach(word => wc(word) = wc.getOrElse(word, 0L) + 1) + wc + }, { case (a, b) => + b.iterator.foreach { case (w: String, cnt: Long) => + a(w) = a.getOrElse(w, 0L) + cnt + } + a + }) + allWords + .toSeq + .sortBy(-_._2) + .take(vocabSize) + .map(_._1) + .zipWithIndex + .toMap + /* + wordCounts + .sortBy(_._2, ascending = false) + .take(vocabSize) + .map(_._1) + .zipWithIndex + .toMap + */ + } + + val documents = tokenized.map { case (id, tokens) => + // Filter tokens by vocabulary, and create word count vector representation of document. + val wc = new mutable.HashMap[Int, Int]() + tokens.foreach { term => + if (vocab.contains(term)) { + val termIndex = vocab(term) + wc(termIndex) = wc.getOrElse(termIndex, 0) + 1 + } + } + val indices = wc.keys.toArray.sorted + val values = indices.map(i => wc(i).toDouble) + + val sb = Vectors.sparse(vocab.size, indices, values) + (id, sb) + } + + val vocabArray = new Array[String](vocab.size) + vocab.foreach { case (term, i) => vocabArray(i) = term } + + (documents, vocabArray) + } +} + +class Tokenizer(sc: SparkContext, stopwordFile: String) extends Serializable { + + private val stopwords: Set[String] = if (stopwordFile.isEmpty) { + Set.empty[String] + } else { + val stopwordText = sc.textFile(stopwordFile).collect() + stopwordText.flatMap(_.stripMargin.split("\\s+")).toSet + } + + // Matches sequences of Unicode letters + private val allWordRegex = "^(\\p{L}*)$".r + + // Ignore words shorter than this length. + private val minWordLength = 3 + + def getWords(text: String): IndexedSeq[String] = { + val words = new mutable.ArrayBuffer[String]() + + // Use Java BreakIterator to tokenize text into words. + val wb = BreakIterator.getWordInstance + wb.setText(text) + + // current,end index start,end of each word + var current = wb.first() + var end = wb.next() + while (end != BreakIterator.DONE) { + // Convert to lowercase + val word: String = text.substring(current, end).toLowerCase + // Remove short words and strings that aren't only letters + word match { + case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => + words += word + case _ => + } + + current = end + end = wb.next() + } + words + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 29b7593df9dc8..73c3b6d205602 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import java.util.Random -import breeze.linalg.{DenseVector => BDV, sum => brzSum, normalize, axpy => brzAxpy} +import breeze.linalg.{DenseVector => BDV, normalize, axpy => brzAxpy} import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi @@ -59,15 +59,15 @@ import org.apache.spark.util.Utils class LDA private ( private var k: Int, private var maxIterations: Int, - private var topicSmoothing: Double, - private var termSmoothing: Double, + private var docConcentration: Double, + private var topicConcentration: Double, private var seed: Long, private var checkpointDir: Option[String], private var checkpointInterval: Int) extends Logging { import LDA._ - def this() = this(k = 10, maxIterations = 20, topicSmoothing = -1, termSmoothing = -1, + def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) /** @@ -85,70 +85,82 @@ class LDA private ( /** * Topic smoothing parameter (commonly named "alpha"). * - * This is the parameter to the Dirichlet prior placed on the per-document topic distributions + * This is the parameter to the Dirichlet prior placed on each document's distribution over topics * ("theta"). We use a symmetric Dirichlet prior. * - * This value should be > 0.0, where larger values mean more smoothing (more regularization). - * If set to -1, then topicSmoothing is set automatically. + * This value should be > 1.0, where larger values mean more smoothing (more regularization). + * If set to -1, then docConcentration is set automatically. * (default = -1 = automatic) * * Automatic setting of parameter: * - For EM: default = (50 / k) + 1. * - The 50/k is common in LDA libraries. * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * + * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), + * but values in (0,1) are not yet supported. */ - def getTopicSmoothing: Double = { - if (this.topicSmoothing > 0) { - this.topicSmoothing - } else { + def getDocConcentration: Double = { + if (this.docConcentration == -1) { (50.0 / k) + 1.0 + } else { + this.docConcentration } } - def setTopicSmoothing(topicSmoothing: Double): this.type = { - require(topicSmoothing > 0.0 || topicSmoothing == -1.0, - s"LDA topicSmoothing must be > 0 (or -1 for auto), but was set to $topicSmoothing") - if (topicSmoothing > 0.0 && topicSmoothing <= 1.0) { - logWarning(s"LDA.topicSmoothing was set to $topicSmoothing, but for EM, we recommend > 1.0") - } - this.topicSmoothing = topicSmoothing + def setDocConcentration(docConcentration: Double): this.type = { + require(docConcentration > 1.0 || docConcentration == -1.0, + s"LDA docConcentration must be > 1.0 (or -1 for auto), but was set to $docConcentration") + this.docConcentration = docConcentration this } + /** Alias for [[getDocConcentration]] */ + def getAlpha: Double = getDocConcentration + + /** Alias for [[setDocConcentration()]] */ + def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) + /** - * Term smoothing parameter (commonly named "eta"). + * Term smoothing parameter (commonly named "beta" or "eta"). * - * This is the parameter to the Dirichlet prior placed on the per-topic term distributions + * This is the parameter to the Dirichlet prior placed on each topic's distribution over terms * (which 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.) + * later papers such as Asuncion et al., 2009). * * This value should be > 0.0. - * If set to -1, then termSmoothing is set automatically. + * If set to -1, then topicConcentration is set automatically. * (default = -1 = automatic) * * Automatic setting of parameter: * - For EM: default = 0.1 + 1. * - The 0.1 gives a small amount of smoothing. * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * + * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), + * but values in (0,1) are not yet supported. */ - def getTermSmoothing: Double = { - if (this.termSmoothing > 0) { - this.termSmoothing - } else { + def getTopicConcentration: Double = { + if (this.topicConcentration == -1) { 1.1 + } else { + this.topicConcentration } } - def setTermSmoothing(termSmoothing: Double): this.type = { - require(termSmoothing > 0.0 || termSmoothing == -1.0, - s"LDA termSmoothing must be > 0 (or -1 for auto), but was set to $termSmoothing") - if (termSmoothing > 0.0 && termSmoothing <= 1.0) { - logWarning(s"LDA.termSmoothing was set to $termSmoothing, but for EM, we recommend > 1.0") - } - this.termSmoothing = termSmoothing + def setTopicConcentration(topicConcentration: Double): this.type = { + require(topicConcentration > 1.0 || topicConcentration == -1.0, + s"LDA topicConcentration must be > 1.0 (or -1 for auto), but was set to $topicConcentration") + this.topicConcentration = topicConcentration this } + /** Alias for [[getTopicConcentration]] */ + def getBeta: Double = getTopicConcentration + + /** Alias for [[setTopicConcentration()]] */ + def setBeta(beta: Double): this.type = setBeta(beta) + /** * Maximum number of iterations for learning. * (default = 20) @@ -200,12 +212,14 @@ class LDA private ( /** * Learn an LDA model using the given dataset. * - * @param documents RDD of documents, where each document is represented as a vector of term - * counts plus an ID. Document IDs must be >= 0. + * @param documents RDD of documents, which are term (word) count vectors paired with IDs. + * The term count vectors are "bags of words" with a fixed-size vocabulary + * (where the vocabulary size is the length of the vector). + * Document IDs must be unique and >= 0. * @return Inferred LDA model */ - def run(documents: RDD[Document]): DistributedLDAModel = { - var state = LDA.initialState(documents, k, getTopicSmoothing, getTermSmoothing, seed, + def run(documents: RDD[(Long, Vector)]): DistributedLDAModel = { + val state = LDA.initialState(documents, k, getDocConcentration, getTopicConcentration, seed, checkpointDir, checkpointInterval) var iter = 0 val iterationTimes = Array.fill[Double](maxIterations)(0) @@ -229,7 +243,7 @@ object LDA { This implementation uses GraphX, where the graph is bipartite with 2 types of vertices: - Document vertices - - indexed {0, 1, ..., numDocuments-1} + - indexed with unique indices >= 0 - Store vectors of length k (# topics). - Term vertices - indexed {-1, -2, ..., -vocabSize} @@ -273,24 +287,6 @@ object LDA { Regularization for Stochastic Matrix Factorization." 2014. */ - /** - * :: DeveloperApi :: - * - * Document with an ID. - * - * @param counts Vector of term (word) counts in the document. - * This is the "bag of words" representation. - * @param id Unique ID associated with this document. - * Documents should be indexed {0, 1, ..., numDocuments-1}. - * - * TODO: Can we remove the id and still be able to zip predicted topics with the Documents? - * - * NOTE: This is currently marked DeveloperApi since it is under active development and may - * undergo API changes. - */ - @DeveloperApi - case class Document(counts: Vector, id: Long) - /** * Vector over topics (length k) of token counts. * The meaning of these counts can vary, and it may or may not be normalized to be a distribution. @@ -315,25 +311,25 @@ object LDA { * data (token counts) in edge descriptors. * @param k Number of topics * @param vocabSize Number of unique terms - * @param topicSmoothing "alpha" - * @param termSmoothing "eta" + * @param docConcentration "alpha" + * @param topicConcentration "beta" or "eta" */ - private[clustering] class LearningState( + private[clustering] class EMOptimizer( var graph: Graph[TopicCounts, TokenCount], val k: Int, val vocabSize: Int, - val topicSmoothing: Double, - val termSmoothing: Double, + val docConcentration: Double, + val topicConcentration: Double, checkpointDir: Option[String], checkpointInterval: Int) { private[LDA] val graphCheckpointer = new PeriodicGraphCheckpointer[TopicCounts, TokenCount]( graph, checkpointDir, checkpointInterval) - def next(): LearningState = { - val eta = termSmoothing + def next(): EMOptimizer = { + val eta = topicConcentration val W = vocabSize - val alpha = topicSmoothing + val alpha = docConcentration val N_k = globalTopicTotals val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = @@ -366,17 +362,7 @@ object LDA { graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) .mapValues(_._2) // Update the vertex descriptors with the new counts. - val newGraph = - GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) - // graph.outerJoinVertices(docTopicDistributions) { (vid, oldDist, newDist) => newDist.get } - /* - previousGraph match { - case Some(prevG) => - prevG.unpersist(blocking = false) - case None => - } - copy(graph = newGraph, previousGraph = Some(graph)) - */ + val newGraph = GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) graph = newGraph graphCheckpointer.updateGraph(newGraph) globalTopicTotals = computeGlobalTopicTotals() @@ -431,22 +417,22 @@ object LDA { * Compute bipartite term/doc graph. */ private def initialState( - docs: RDD[Document], + docs: RDD[(Long, Vector)], k: Int, - topicSmoothing: Double, - termSmoothing: Double, + docConcentration: Double, + topicConcentration: Double, randomSeed: Long, checkpointDir: Option[String], - checkpointInterval: Int): LearningState = { + checkpointInterval: Int): EMOptimizer = { // For each document, create an edge (Document -> Term) for each unique term in the document. - val edges: RDD[Edge[TokenCount]] = docs.flatMap { doc => + val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => // Add edges for terms with non-zero counts. - doc.counts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => - Edge(doc.id, term2index(term), cnt) + termCounts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + Edge(docID, term2index(term), cnt) } } - val vocabSize = docs.take(1).head.counts.size + val vocabSize = docs.take(1).head._2.size // Create vertices. // Initially, we use random soft assignments of tokens to topics (random gamma). @@ -480,7 +466,7 @@ object LDA { val graph = Graph(docVertices ++ termVertices, edges) .partitionBy(PartitionStrategy.EdgePartition1D) - new LearningState(graph, k, vocabSize, topicSmoothing, termSmoothing, checkpointDir, + new EMOptimizer(graph, k, vocabSize, docConcentration, topicConcentration, checkpointDir, checkpointInterval) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 279e6ed99c71e..6a1212a114e5c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -121,25 +121,29 @@ abstract class LDAModel private[clustering] { * - Even with the prior, this is NOT the same as the data log likelihood given the * hyperparameters. * - * @param documents A set of documents, where each is represented as a vector of term counts. + * @param documents RDD of documents, which are term (word) count vectors paired with IDs. + * The term count vectors are "bags of words" with a fixed-size vocabulary + * (where the vocabulary size is the length of the vector). * This must use the same vocabulary (ordering of term counts) as in training. - * Document IDs must be >= 0. + * Document IDs must be unique and >= 0. * @return Estimated log likelihood of the data under this model */ - // def logLikelihood(documents: RDD[Document]): Double + // def logLikelihood(documents: RDD[(Long, Vector)]): Double /* TODO * Compute the estimated topic distribution for each document. * This is often called “theta” in the literature. * - * @param documents A set of documents, where each is represented as a vector of term counts. + * @param documents RDD of documents, which are term (word) count vectors paired with IDs. + * The term count vectors are "bags of words" with a fixed-size vocabulary + * (where the vocabulary size is the length of the vector). * This must use the same vocabulary (ordering of term counts) as in training. - * Document IDs must be >= 0. + * Document IDs must be unique and >= 0. * @return Estimated topic distribution for each document. * The returned RDD may be zipped with the given RDD, where each returned vector * is a multinomial distribution over topics. */ - // def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] + // def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] } @@ -177,10 +181,10 @@ class LocalLDAModel private[clustering] ( } // TODO - // override def logLikelihood(documents: RDD[Document]): Double = ??? + // override def logLikelihood(documents: RDD[(Long, Vector)]): Double = ??? // TODO: - // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? + // override def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = ??? } @@ -201,15 +205,15 @@ class DistributedLDAModel private ( private val globalTopicTotals: LDA.TopicCounts, val k: Int, val vocabSize: Int, - private val topicSmoothing: Double, - private val termSmoothing: Double, + private val docConcentration: Double, + private val topicConcentration: Double, private[spark] val iterationTimes: Array[Double]) extends LDAModel { import LDA._ - private[clustering] def this(state: LDA.LearningState, iterationTimes: Array[Double]) = { - this(state.graph, state.globalTopicTotals, state.k, state.vocabSize, state.topicSmoothing, - state.termSmoothing, iterationTimes) + private[clustering] def this(state: LDA.EMOptimizer, iterationTimes: Array[Double]) = { + this(state.graph, state.globalTopicTotals, state.k, state.vocabSize, state.docConcentration, + state.topicConcentration, iterationTimes) } /** @@ -270,7 +274,7 @@ class DistributedLDAModel private ( } // TODO - // override def logLikelihood(documents: RDD[Document]): Double = ??? + // override def logLikelihood(documents: RDD[(Long, Vector)]): Double = ??? /** * Log likelihood of the observed tokens in the training set, @@ -283,8 +287,8 @@ class DistributedLDAModel private ( * hyperparameters. */ lazy val logLikelihood: Double = { - val eta = termSmoothing - val alpha = topicSmoothing + val eta = topicConcentration + val alpha = docConcentration assert(eta > 1.0) assert(alpha > 1.0) val N_k = globalTopicTotals @@ -308,8 +312,8 @@ class DistributedLDAModel private ( * log P(topics, topic distributions for docs | alpha, eta) */ lazy val logPrior: Double = { - val eta = termSmoothing - val alpha = topicSmoothing + val eta = topicConcentration + val alpha = docConcentration // Term vertices: Compute phi_{wk}. Use to compute prior log probability. // Doc vertex: Compute theta_{kj}. Use to compute prior log probability. val N_k = globalTopicTotals @@ -344,6 +348,6 @@ class DistributedLDAModel private ( } // TODO: - // override def topicDistributions(documents: RDD[Document]): RDD[(Long, Vector)] = ??? + // override def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = ??? } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 1aaabe44a6be8..a1aa64dbe3d3a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.mllib.clustering +import org.scalatest.FunSuite + import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ -import org.scalatest.FunSuite class LDASuite extends FunSuite with MLlibTestSparkContext { @@ -57,8 +58,8 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { // Train a model val lda = new LDA() lda.setK(k) - lda.setTopicSmoothing(topicSmoothing) - lda.setTermSmoothing(termSmoothing) + lda.setDocConcentration(topicSmoothing) + lda.setTopicConcentration(termSmoothing) lda.setMaxIterations(5) lda.setSeed(12345) val corpus = sc.parallelize(tinyCorpus, 2) @@ -91,7 +92,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { val topicDistributions = model.topicDistributions.collect() // Ensure all documents are covered. assert(topicDistributions.size === tinyCorpus.size) - assert(tinyCorpus.map(_.id).toSet === topicDistributions.map(_._1).toSet) + assert(tinyCorpus.map(_._1).toSet === topicDistributions.map(_._1).toSet) // Ensure we have proper distributions topicDistributions.foreach { case (docId, topicDistribution) => assert(topicDistribution.size === tinyK) @@ -127,8 +128,8 @@ private object LDASuite { ) def tinyTopics: Matrix = new DenseMatrix(numRows = tinyVocabSize, numCols = tinyK, values = tinyTopicsAsArray.fold(Array.empty[Double])(_ ++ _)) - def tinyTopicDescription: Array[Array[(Double, String)]] = tinyTopicsAsArray.map { topic => - topic.zipWithIndex.sortBy(-_._1).map { case (weight, term) => (weight, term.toString) } + def tinyTopicDescription: Array[Array[(Double, Int)]] = tinyTopicsAsArray.map { topic => + topic.zipWithIndex.sortBy(-_._1) } def tinyCorpus = Array( @@ -137,7 +138,7 @@ private object LDASuite { Vectors.dense(2, 3, 12, 3, 1), Vectors.dense(0, 3, 1, 9, 8), Vectors.dense(1, 1, 4, 2, 6) - ).zipWithIndex.map { case (wordCounts, docId) => LDA.Document(wordCounts, docId) } - assert(tinyCorpus.forall(_.counts.size == tinyVocabSize)) // sanity check for test data + ).zipWithIndex.map { case (wordCounts, docId) => (docId.toLong, wordCounts) } + assert(tinyCorpus.forall(_._2.size == tinyVocabSize)) // sanity check for test data } From b75472d8ed865276f2e117165bdb19aa01fc0627 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 30 Jan 2015 10:16:50 -0800 Subject: [PATCH 21/30] merged improvements from LDATiming into LDAExample. Will remove LDATiming after done testing --- .../spark/examples/mllib/LDAExample.scala | 88 +++++++++++++------ .../spark/examples/mllib/LDATiming.scala | 65 ++++---------- 2 files changed, 78 insertions(+), 75 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 8053761a802db..cf756e0dc05d1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -17,7 +17,7 @@ package org.apache.spark.examples.mllib -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable import java.text.BreakIterator @@ -47,7 +47,9 @@ object LDAExample { docConcentration: Double = -1, topicConcentration: Double = -1, vocabSize: Int = 10000, - stopwordFile: String = "") extends AbstractParams[Params] + stopwordFile: String = "", + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() @@ -69,15 +71,25 @@ object LDAExample { s" default: ${defaultParams.topicConcentration}") .action((x, c) => c.copy(topicConcentration = x)) opt[Int]("vocabSize") - .text(s"number of distinct word types to use, chosen by frequency." + + .text(s"number of distinct word types to use, chosen by frequency. (-1=all)" + s" default: ${defaultParams.vocabSize}") .action((x, c) => c.copy(vocabSize = x)) opt[String]("stopwordFile") .text(s"filepath for a list of stopwords. Note: This must fit on a single machine." + s" default: ${defaultParams.stopwordFile}") .action((x, c) => c.copy(stopwordFile = x)) + opt[String]("checkpointDir") + .text(s"Directory for checkpointing intermediate results." + + s" Checkpointing helps with recovery and eliminates temporary shuffle files on disk." + + s" default: ${defaultParams.checkpointDir}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"Iterations between each checkpoint. Only used if checkpointDir is set." + + s" default: ${defaultParams.checkpointInterval}") + .action((x, c) => c.copy(checkpointInterval = x)) arg[String]("...") - .text("input paths (directories) to plain text corpora") + .text("input paths (directories) to plain text corpora." + + " Each text file line should hold 1 document.") .unbounded() .required() .action((x, c) => c.copy(input = c.input :+ x)) @@ -98,8 +110,21 @@ object LDAExample { Logger.getRootLogger.setLevel(Level.WARN) // Load documents, and prepare them for LDA. + val preprocessStart = System.nanoTime() val (corpus, vocabArray) = preprocess(sc, params.input, params.vocabSize, params.stopwordFile) - corpus.cache() // cache since LDA is iterative + corpus.cache() + val actualCorpusSize = corpus.count() + val actualVocabSize = vocabArray.size + val actualNumTokens = corpus.map(_._2.toArray.sum.toLong).sum().toLong + val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 + + println() + println(s"Corpus summary:") + println(s"\t Training set size: $actualCorpusSize documents") + println(s"\t Vocabulary size: $actualVocabSize terms") + println(s"\t Training set size: $actualNumTokens tokens") + println(s"\t Preprocessing time: $preprocessElapsed sec") + println() // Run LDA. val lda = new LDA() @@ -107,15 +132,18 @@ object LDAExample { .setMaxIterations(params.maxIterations) .setDocConcentration(params.docConcentration) .setTopicConcentration(params.topicConcentration) + .setCheckpointInterval(params.checkpointInterval) + if (params.checkpointDir.nonEmpty) { + lda.setCheckpointDir(params.checkpointDir.get) + } val startTime = System.nanoTime() val ldaModel = lda.run(corpus) val elapsed = (System.nanoTime() - startTime) / 1e9 println(s"Finished training LDA model. Summary:") println(s"\t Training time: $elapsed sec") - println(s"\t Training set size: ${corpus.count()} documents") - println(s"\t Vocabulary size: ${vocabArray.size} terms") - println(s"\t Training data average log likelihood: ${ldaModel.logLikelihood / corpus.count()}") + val avgLogLikelihood = ldaModel.logLikelihood / actualCorpusSize.toDouble + println(s"\t Training data average log likelihood: $avgLogLikelihood") println() // Print the topics, showing the top-weighted terms for each topic. @@ -143,37 +171,41 @@ object LDAExample { vocabSize: Int, stopwordFile: String): (RDD[(Long, Vector)], Array[String]) = { - val files: Seq[RDD[(String, String)]] = for (p <- paths) yield { - sc.wholeTextFiles(p) + // Get dataset of document texts + // One document per line in each text file. + val files: Seq[RDD[String]] = for (p <- paths) yield { + sc.textFile(p) } - - // Dataset of document texts - val textRDD: RDD[String] = - files.reduce(_ ++ _) // combine results from multiple paths - .map { case (path, text) => text } + val textRDD: RDD[String] = files.reduce(_ ++ _) // combine results from multiple paths // Split text into words val tokenizer = new SimpleTokenizer(sc, stopwordFile) val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => id -> tokenizer.getWords(text) } - - // Counts words: RDD[(word, wordCount)] - val wordCounts: RDD[(String, Int)] = tokenized - .flatMap { case (_, tokens) => tokens.map(_ -> 1) } - .reduceByKey(_ + _) + tokenized.cache() // Choose vocabulary: Map[word -> id] - val vocab: Map[String, Int] = wordCounts - .sortBy(_._2, ascending = false) - .take(vocabSize) - .map(_._1) - .zipWithIndex - .toMap + val vocab: Map[String, Int] = { + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Int)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1) } + .reduceByKey(_ + _) + val sortedWC = wordCounts + .sortBy(_._2, ascending = false) + val selectedWC = if (vocabSize == -1) { + sortedWC.collect() + } else { + sortedWC.take(vocabSize) + } + selectedWC.map(_._1) + .zipWithIndex + .toMap + } val documents = tokenized.map { case (id, tokens) => // Filter tokens by vocabulary, and create word count vector representation of document. - val wc = new scala.collection.mutable.HashMap[Int, Int]() + val wc = new mutable.HashMap[Int, Int]() tokens.foreach { term => if (vocab.contains(term)) { val termIndex = vocab(term) @@ -216,7 +248,7 @@ private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Se def getWords(text: String): IndexedSeq[String] = { - val words = new ArrayBuffer[String]() + val words = new mutable.ArrayBuffer[String]() // Use Java BreakIterator to tokenize text into words. val wb = BreakIterator.getWordInstance diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala index 406dc31b392c3..75b17d35463d3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala @@ -29,7 +29,7 @@ import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.mllib.clustering.LDA import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.OpenHashSet + /** * An app for timing Latent Dirichlet Allocation (LDA). @@ -55,8 +55,8 @@ object LDATiming { ks: Array[Int] = Array(20), numPartitions: Int = 16, maxIterations: Int = 10, - docConcentration: Double = 0.1, - topicConcentration: Double = 0.1, + docConcentration: Double = -1, + topicConcentration: Double = -1, vocabSizes: Array[Int] = Array(10000), stopwordFile: String = "", checkpointDir: Option[String] = None, @@ -129,7 +129,7 @@ object LDATiming { val preprocessStart = System.nanoTime() val (corpus, vocabArray) = preprocess(sc, params.input, corpusSize, vocabSize, params.stopwordFile) - corpus.repartition(params.numPartitions).cache() // cache since LDA is iterative + corpus.repartition(params.numPartitions).cache() val actualCorpusSize = corpus.count() val actualVocabSize = vocabArray.size val actualNumTokens = corpus.map(_._2.toArray.sum.toLong).sum().toLong @@ -141,6 +141,7 @@ object LDATiming { println(s"\t Vocabulary size: $actualVocabSize terms") println(s"\t Training set size: $actualNumTokens tokens") println(s"\t Preprocessing time: $preprocessElapsed sec") + println() for (k <- params.ks) { // Run LDA. @@ -222,54 +223,24 @@ object LDATiming { val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => id -> tokenizer.getWords(text) } - - /* - // Counts words: RDD[(word, wordCount)] - val wordCounts: RDD[(String, Int)] = tokenized - .flatMap { case (_, tokens) => tokens.map(_ -> 1) } - .reduceByKey(_ + _) - */ + tokenized.cache() // Choose vocabulary: Map[word -> id] - val vocab: Map[String, Int] = if (vocabSize == -1) { - val allWords = tokenized.aggregate(new OpenHashSet[String])({ - case (wordSet, (docId, words)) => - words.foreach(word => wordSet.add(word)) - wordSet - }, { case (a, b) => - b.iterator.foreach(w => a.add(w)) - a - }) - allWords - .iterator - .zipWithIndex - .toMap - } else { - val allWords = tokenized.aggregate(new mutable.HashMap[String, Long])({ - case (wc, (docId, words)) => - words.foreach(word => wc(word) = wc.getOrElse(word, 0L) + 1) - wc - }, { case (a, b) => - b.iterator.foreach { case (w: String, cnt: Long) => - a(w) = a.getOrElse(w, 0L) + cnt - } - a - }) - allWords - .toSeq - .sortBy(-_._2) - .take(vocabSize) - .map(_._1) - .zipWithIndex - .toMap - /* - wordCounts + val vocab: Map[String, Int] = { + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Int)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1) } + .reduceByKey(_ + _) + val sortedWC = wordCounts .sortBy(_._2, ascending = false) - .take(vocabSize) - .map(_._1) + val selectedWC = if (vocabSize == -1) { + sortedWC.collect() + } else { + sortedWC.take(vocabSize) + } + selectedWC.map(_._1) .zipWithIndex .toMap - */ } val documents = tokenized.map { case (id, tokens) => From 91aadfe8944e688990678285a62ea6cab4ad05c5 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 30 Jan 2015 12:23:57 -0800 Subject: [PATCH 22/30] Added Java-friendly run method to LDA. Added Java test suite for LDA. Changed LDAModel.describeTopics to return Java-friendly type --- .../spark/examples/mllib/LDAExample.scala | 6 +- .../spark/examples/mllib/LDATiming.scala | 6 +- .../apache/spark/mllib/clustering/LDA.scala | 24 ++-- .../spark/mllib/clustering/LDAModel.scala | 62 +++++---- .../spark/mllib/clustering/JavaLDASuite.java | 118 ++++++++++++++++++ .../spark/mllib/clustering/LDASuite.scala | 39 +++--- 6 files changed, 198 insertions(+), 57 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index cf756e0dc05d1..8baf67da71414 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -148,13 +148,13 @@ object LDAExample { // Print the topics, showing the top-weighted terms for each topic. val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) - val topics = topicIndices.map { topic => - topic.map { case (weight, term) => (weight, vocabArray(term.toInt)) } + val topics = topicIndices.map { case (terms, termWeights) => + terms.zip(termWeights).map { case (term, weight) => (vocabArray(term.toInt), weight) } } println(s"${params.k} topics:") topics.zipWithIndex.foreach { case (topic, i) => println(s"TOPIC $i") - topic.foreach { case (weight, term) => + topic.foreach { case (term, weight) => println(s"$term\t$weight") } println() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala index 75b17d35463d3..82c524c8fa16b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala @@ -167,13 +167,13 @@ object LDATiming { // Print the topics, showing the top-weighted terms for each topic. val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) - val topics = topicIndices.map { topic => - topic.map { case (weight, term) => (weight, vocabArray(term.toInt))} + val topics = topicIndices.map { case (terms, termWeights) => + terms.zip(termWeights).map { case (term, weight) => (vocabArray(term.toInt), weight) } } println(s"$k topics:") topics.zipWithIndex.foreach { case (topic, i) => println(s"TOPIC $i") - topic.foreach { case (weight, term) => + topic.foreach { case (term, weight) => println(s"$term\t$weight") } println() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 73c3b6d205602..1755b995d57e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -23,6 +23,7 @@ import breeze.linalg.{DenseVector => BDV, normalize, axpy => brzAxpy} import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer @@ -233,10 +234,15 @@ class LDA private ( state.graphCheckpointer.deleteAllCheckpoints() new DistributedLDAModel(state, iterationTimes) } + + /** Java-friendly version of [[run()]] */ + def run(documents: JavaRDD[(java.lang.Long, Vector)]): DistributedLDAModel = { + run(documents.rdd.map(id_counts => (id_counts._1.asInstanceOf[Long], id_counts._2))) + } } -object LDA { +private[clustering] object LDA { /* DEVELOPERS NOTE: @@ -291,18 +297,18 @@ object LDA { * Vector over topics (length k) of token counts. * The meaning of these counts can vary, and it may or may not be normalized to be a distribution. */ - private[clustering] type TopicCounts = BDV[Double] + type TopicCounts = BDV[Double] - private[clustering] type TokenCount = Double + type TokenCount = Double /** Term vertex IDs are {-1, -2, ..., -vocabSize} */ - private[clustering] def term2index(term: Int): Long = -(1 + term.toLong) + def term2index(term: Int): Long = -(1 + term.toLong) - private[clustering] def index2term(termIndex: Long): Int = -(1 + termIndex).toInt + def index2term(termIndex: Long): Int = -(1 + termIndex).toInt - private[clustering] def isDocumentVertex(v: (VertexId, _)): Boolean = v._1 >= 0 + def isDocumentVertex(v: (VertexId, _)): Boolean = v._1 >= 0 - private[clustering] def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 + def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 /** * State for EM algorithm: data + parameter graph, plus algorithm parameters. @@ -314,7 +320,7 @@ object LDA { * @param docConcentration "alpha" * @param topicConcentration "beta" or "eta" */ - private[clustering] class EMOptimizer( + class EMOptimizer( var graph: Graph[TopicCounts, TokenCount], val k: Int, val vocabSize: Int, @@ -374,7 +380,7 @@ object LDA { * * Note: This executes an action on the graph RDDs. */ - private[clustering] var globalTopicTotals: TopicCounts = computeGlobalTopicTotals() + var globalTopicTotals: TopicCounts = computeGlobalTopicTotals() private def computeGlobalTopicTotals(): TopicCounts = { val numTopics = k diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 6a1212a114e5c..4552d75acf849 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -60,10 +60,10 @@ abstract class LDAModel private[clustering] { * * @param maxTermsPerTopic Maximum number of terms to collect for each topic. * @return Array over topics, where each element is a set of top terms represented - * as (term weight in topic, term index). + * as (term index, term weight in topic). * Each topic's terms are sorted in order of decreasing weight. */ - def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] + def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] /** * Return the topics described by weighted terms. @@ -71,10 +71,10 @@ abstract class LDAModel private[clustering] { * WARNING: If vocabSize and k are large, this can return a large object! * * @return Array over topics, where each element is a set of top terms represented - * as (term weight in topic, term index). + * as (term index, term weight in topic). * Each topic's terms are sorted in order of decreasing weight. */ - def describeTopics(): Array[Array[(Double, Int)]] = describeTopics(vocabSize) + def describeTopics(): Array[(Array[Int], Array[Double])] = describeTopics(vocabSize) /* TODO (once LDA can be trained with Strings or given a dictionary) * Return the topics described by weighted terms. @@ -89,11 +89,11 @@ abstract class LDAModel private[clustering] { * * @param maxTermsPerTopic Maximum number of terms to collect for each topic. * @return Array over topics, where each element is a set of top terms represented - * as (term weight in topic, term), where "term" is either the actual term text + * as (term, term weight in topic), where "term" is either the actual term text * (if available) or the term index. * Each topic's terms are sorted in order of decreasing weight. */ - //def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[Array[(Double, String)]] + //def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[(Array[Double], Array[String])] /* TODO (once LDA can be trained with Strings or given a dictionary) * Return the topics described by weighted terms. @@ -105,11 +105,11 @@ abstract class LDAModel private[clustering] { * WARNING: If vocabSize and k are large, this can return a large object! * * @return Array over topics, where each element is a set of top terms represented - * as (term weight in topic, term), where "term" is either the actual term text + * as (term, term weight in topic), where "term" is either the actual term text * (if available) or the term index. * Each topic's terms are sorted in order of decreasing weight. */ - //def describeTopicsAsStrings(): Array[Array[(Double, String)]] = + //def describeTopicsAsStrings(): Array[(Array[Double], Array[String])] = // describeTopicsAsStrings(vocabSize) /* TODO @@ -172,11 +172,13 @@ class LocalLDAModel private[clustering] ( override def topicsMatrix: Matrix = topics - override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { + override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { val brzTopics = topics.toBreeze.toDenseMatrix Range(0, k).map { topicIndex => val topic = normalize(brzTopics(::, topicIndex), 1.0) - topic.toArray.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic) + val (termWeights, terms) = + topic.toArray.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic).unzip + (terms.toArray, termWeights.toArray) }.toArray } @@ -248,29 +250,35 @@ class DistributedLDAModel private ( Matrices.fromBreeze(brzTopics) } - override def describeTopics(maxTermsPerTopic: Int): Array[Array[(Double, Int)]] = { + override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { val numTopics = k // Note: N_k is not needed to find the top terms, but it is needed to normalize weights // to a distribution over terms. val N_k: TopicCounts = globalTopicTotals - graph.vertices.filter(isTermVertex) - .mapPartitions { termVertices => - // For this partition, collect the most common terms for each topic in queues: - // queues(topic) = queue of (term weight, term index). - // Term weights are N_{wk} / N_k. - val queues = Array.fill(numTopics)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) - for ((termId, n_wk) <- termVertices) { - var topic = 0 - while (topic < numTopics) { - queues(topic) += (n_wk(topic) / N_k(topic) -> index2term(termId.toInt)) - topic += 1 + val topicsInQueues: Array[BoundedPriorityQueue[(Double, Int)]] = + graph.vertices.filter(isTermVertex) + .mapPartitions { termVertices => + // For this partition, collect the most common terms for each topic in queues: + // queues(topic) = queue of (term weight, term index). + // Term weights are N_{wk} / N_k. + val queues = + Array.fill(numTopics)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) + for ((termId, n_wk) <- termVertices) { + var topic = 0 + while (topic < numTopics) { + queues(topic) += (n_wk(topic) / N_k(topic) -> index2term(termId.toInt)) + topic += 1 + } } + Iterator(queues) + }.reduce { (q1, q2) => + q1.zip(q2).foreach { case (a, b) => a ++= b} + q1 } - Iterator(queues) - }.reduce { (q1, q2) => - q1.zip(q2).foreach { case (a, b) => a ++= b} - q1 - }.map(_.toArray.sortBy(-_._1)) + topicsInQueues.map { q => + val (termWeights, terms) = q.toArray.sortBy(-_._1).unzip + (terms.toArray, termWeights.toArray) + } } // TODO diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java new file mode 100644 index 0000000000000..9fc561bb3b332 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -0,0 +1,118 @@ +/* + * 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.mllib.clustering; + +import java.io.Serializable; +import java.util.ArrayList; + +import scala.Tuple2; + +import org.junit.After; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; + + +public class JavaLDASuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLDA"); + tinyCorpus = new ArrayList>(); + for (int i = 0; i < LDASuite$.MODULE$.tinyCorpus().length; i++) { + tinyCorpus.add(new Tuple2((Long)LDASuite$.MODULE$.tinyCorpus()[i]._1(), + LDASuite$.MODULE$.tinyCorpus()[i]._2())); + } + corpus = sc.parallelize(tinyCorpus, 2); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void localLDAModel() { + LocalLDAModel model = new LocalLDAModel(LDASuite$.MODULE$.tinyTopics()); + + // Check: basic parameters + assertEquals(model.k(), tinyK); + assertEquals(model.vocabSize(), tinyVocabSize); + assertEquals(model.topicsMatrix(), tinyTopics); + + // Check: describeTopics() with all terms + Tuple2[] fullTopicSummary = model.describeTopics(); + assertEquals(fullTopicSummary.length, tinyK); + for (int i = 0; i < fullTopicSummary.length; i++) { + assertArrayEquals(fullTopicSummary[i]._1(), tinyTopicDescription[i]._1()); + assertArrayEquals(fullTopicSummary[i]._2(), tinyTopicDescription[i]._2(), 1e-5); + } + } + + @Test + public void distributedLDAModel() { + int k = 3; + double topicSmoothing = 1.2; + double termSmoothing = 1.2; + + // Train a model + LDA lda = new LDA(); + lda.setK(k) + .setDocConcentration(topicSmoothing) + .setTopicConcentration(termSmoothing) + .setMaxIterations(5) + .setSeed(12345); + + DistributedLDAModel model = lda.run(corpus); + + // Check: basic parameters + LocalLDAModel localModel = model.toLocal(); + assertEquals(model.k(), k); + assertEquals(localModel.k(), k); + assertEquals(model.vocabSize(), tinyVocabSize); + assertEquals(localModel.vocabSize(), tinyVocabSize); + assertEquals(model.topicsMatrix(), localModel.topicsMatrix()); + + // Check: topic summaries + Tuple2[] roundedTopicSummary = model.describeTopics(); + assertEquals(roundedTopicSummary.length, k); + Tuple2[] roundedLocalTopicSummary = localModel.describeTopics(); + assertEquals(roundedLocalTopicSummary.length, k); + + // Check: log probabilities + assert(model.logLikelihood() < 0.0); + assert(model.logPrior() < 0.0); + } + + private static int tinyK = LDASuite$.MODULE$.tinyK(); + private static int tinyVocabSize = LDASuite$.MODULE$.tinyVocabSize(); + private static Matrix tinyTopics = LDASuite$.MODULE$.tinyTopics(); + private static Tuple2[] tinyTopicDescription = + LDASuite$.MODULE$.tinyTopicDescription(); + private ArrayList> tinyCorpus; + JavaRDD> corpus; + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index a1aa64dbe3d3a..302d751eb8a94 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -38,15 +38,19 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { // Check: describeTopics() with all terms val fullTopicSummary = model.describeTopics() assert(fullTopicSummary.size === tinyK) - fullTopicSummary.zip(tinyTopicDescription).foreach { case (algSummary, tinySummary) => - assert(algSummary === tinySummary) + fullTopicSummary.zip(tinyTopicDescription).foreach { + case ((algTerms, algTermWeights), (terms, termWeights)) => + assert(algTerms === terms) + assert(algTermWeights === termWeights) } // Check: describeTopics() with some terms val smallNumTerms = 3 val smallTopicSummary = model.describeTopics(maxTermsPerTopic = smallNumTerms) - smallTopicSummary.zip(tinyTopicDescription).foreach { case (algSummary, tinySummary) => - assert(algSummary === tinySummary.slice(0, smallNumTerms)) + smallTopicSummary.zip(tinyTopicDescription).foreach { + case ((algTerms, algTermWeights), (terms, termWeights)) => + assert(algTerms === terms.slice(0, smallNumTerms)) + assert(algTermWeights === termWeights.slice(0, smallNumTerms)) } } @@ -58,10 +62,10 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { // Train a model val lda = new LDA() lda.setK(k) - lda.setDocConcentration(topicSmoothing) - lda.setTopicConcentration(termSmoothing) - lda.setMaxIterations(5) - lda.setSeed(12345) + .setDocConcentration(topicSmoothing) + .setTopicConcentration(termSmoothing) + .setMaxIterations(5) + .setSeed(12345) val corpus = sc.parallelize(tinyCorpus, 2) val model: DistributedLDAModel = lda.run(corpus) @@ -76,13 +80,17 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { // Check: topic summaries // The odd decimal formatting and sorting is a hack to do a robust comparison. - val roundedTopicSummary = model.describeTopics().map { case topic => + val roundedTopicSummary = model.describeTopics().map { case (terms, termWeights) => // cut values to 3 digits after the decimal place - topic.map { case (weight, term) => ("%.3f".format(weight).toDouble, term.toInt)} + terms.zip(termWeights).map { case (term, weight) => + ("%.3f".format(weight).toDouble, term.toInt) + } }.sortBy(_.mkString("")) - val roundedLocalTopicSummary = localModel.describeTopics().map { case topic => + val roundedLocalTopicSummary = localModel.describeTopics().map { case (terms, termWeights) => // cut values to 3 digits after the decimal place - topic.map { case (weight, term) => ("%.3f".format(weight).toDouble, term.toInt)} + terms.zip(termWeights).map { case (term, weight) => + ("%.3f".format(weight).toDouble, term.toInt) + } }.sortBy(_.mkString("")) roundedTopicSummary.zip(roundedLocalTopicSummary).foreach { case (t1, t2) => assert(t1 === t2) @@ -117,7 +125,7 @@ class LDASuite extends FunSuite with MLlibTestSparkContext { } } -private object LDASuite { +private[clustering] object LDASuite { def tinyK: Int = 3 def tinyVocabSize: Int = 5 @@ -128,8 +136,9 @@ private object LDASuite { ) def tinyTopics: Matrix = new DenseMatrix(numRows = tinyVocabSize, numCols = tinyK, values = tinyTopicsAsArray.fold(Array.empty[Double])(_ ++ _)) - def tinyTopicDescription: Array[Array[(Double, Int)]] = tinyTopicsAsArray.map { topic => - topic.zipWithIndex.sortBy(-_._1) + def tinyTopicDescription: Array[(Array[Int], Array[Double])] = tinyTopicsAsArray.map { topic => + val (termWeights, terms) = topic.zipWithIndex.sortBy(-_._1).unzip + (terms.toArray, termWeights.toArray) } def tinyCorpus = Array( From 1a231b484f5057ac65b06b053fda104c7bd2d1e4 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 30 Jan 2015 12:28:18 -0800 Subject: [PATCH 23/30] fixed scalastyle --- .../org/apache/spark/examples/mllib/LDATiming.scala | 13 +++++++++---- .../apache/spark/mllib/clustering/LDAModel.scala | 4 ++-- .../mllib/impl/PeriodicGraphCheckpointer.scala | 7 ++++--- 3 files changed, 15 insertions(+), 9 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala index 82c524c8fa16b..445a8603215cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala @@ -103,7 +103,8 @@ object LDATiming { s" default: ${defaultParams.checkpointInterval}") .action((x, c) => c.copy(checkpointInterval = x)) arg[String]("...") - .text("input paths (directories) to plain text corpora. Each text file line should hold 1 document.") + .text("input paths (directories) to plain text corpora." + + " Each text file line should hold 1 document.") .unbounded() .required() .action((x, c) => c.copy(input = c.input :+ x)) @@ -128,7 +129,8 @@ object LDATiming { for (vocabSize <- params.vocabSizes) { val preprocessStart = System.nanoTime() - val (corpus, vocabArray) = preprocess(sc, params.input, corpusSize, vocabSize, params.stopwordFile) + val (corpus, vocabArray) = preprocess(sc, params.input, corpusSize, vocabSize, + params.stopwordFile) corpus.repartition(params.numPartitions).cache() val actualCorpusSize = corpus.count() val actualVocabSize = vocabArray.size @@ -162,7 +164,8 @@ object LDATiming { println(s"\t Training time: $elapsed sec") val avgLogLikelihood = ldaModel.logLikelihood / actualCorpusSize.toDouble println(s"\t Training data average log likelihood: $avgLogLikelihood") - println(s"\t Training times per iteration (sec):\n${ldaModel.iterationTimes.mkString("\t", "\n\t", "\n")}") + println(s"\t Training times per iteration (sec):\n" + + s"${ldaModel.iterationTimes.mkString("\t", "\n\t", "\n")}") println() // Print the topics, showing the top-weighted terms for each topic. @@ -181,7 +184,9 @@ object LDATiming { println() println("--------------------------------------------------------------") println() - println(s"RESULTS: $corpusSize $vocabSize $actualCorpusSize $actualVocabSize $actualNumTokens $k $elapsed $avgLogLikelihood ${ldaModel.iterationTimes.mkString(" ")}") + println(s"RESULTS: $corpusSize $vocabSize $actualCorpusSize $actualVocabSize" + + s" $actualNumTokens $k $elapsed $avgLogLikelihood" + + s" ${ldaModel.iterationTimes.mkString(" ")}") println() println("==============================================================") println() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 4552d75acf849..aade713753527 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -93,7 +93,7 @@ abstract class LDAModel private[clustering] { * (if available) or the term index. * Each topic's terms are sorted in order of decreasing weight. */ - //def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[(Array[Double], Array[String])] + // def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[(Array[Double], Array[String])] /* TODO (once LDA can be trained with Strings or given a dictionary) * Return the topics described by weighted terms. @@ -109,7 +109,7 @@ abstract class LDAModel private[clustering] { * (if available) or the term index. * Each topic's terms are sorted in order of decreasing weight. */ - //def describeTopicsAsStrings(): Array[(Array[Double], Array[String])] = + // def describeTopicsAsStrings(): Array[(Array[Double], Array[String])] = // describeTopicsAsStrings(vocabSize) /* TODO diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala index 56259cddd5e51..864829e03849f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -50,10 +50,10 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( val checkpointDir: Option[String], val checkpointInterval: Int) extends Logging { - /** FIFO queue of past checkpointed RDDs*/ + /** FIFO queue of past checkpointed RDDs */ private val checkpointQueue = mutable.Queue[Graph[VD, ED]]() - /** FIFO queue of past persisted RDDs*/ + /** FIFO queue of past persisted RDDs */ private val persistedQueue = mutable.Queue[Graph[VD, ED]]() /** Number of times [[updateGraph()]] has been called */ @@ -91,7 +91,8 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( // before the graph has been materialized. while (persistedQueue.size > 3) { val graphToUnpersist = persistedQueue.dequeue() - println(s"PeriodicGraphCheckpointer.updateGraph: unpersisting ${graphToUnpersist.vertices.id}") + println(s"PeriodicGraphCheckpointer.updateGraph:" + + s" unpersisting ${graphToUnpersist.vertices.id}") graphToUnpersist.unpersist(blocking = false) } updateCount += 1 From e8d8acfaf5696f9e2e23edf7fe0880afa6166141 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 30 Jan 2015 15:28:06 -0800 Subject: [PATCH 24/30] Added catch for BreakIterator exception. Improved preprocessing to reduce passes over data --- .../spark/examples/mllib/LDAExample.scala | 49 ++++++++++-------- .../spark/examples/mllib/LDATiming.scala | 50 +++++++++++-------- 2 files changed, 58 insertions(+), 41 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 8baf67da71414..5d65b0f33c1fe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -111,11 +111,11 @@ object LDAExample { // Load documents, and prepare them for LDA. val preprocessStart = System.nanoTime() - val (corpus, vocabArray) = preprocess(sc, params.input, params.vocabSize, params.stopwordFile) + val (corpus, vocabArray, actualNumTokens) = + preprocess(sc, params.input, params.vocabSize, params.stopwordFile) corpus.cache() val actualCorpusSize = corpus.count() val actualVocabSize = vocabArray.size - val actualNumTokens = corpus.map(_._2.toArray.sum.toLong).sum().toLong val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 println() @@ -164,12 +164,13 @@ object LDAExample { /** * Load documents, tokenize them, create vocabulary, and prepare documents as term count vectors. + * @return (corpus, vocabulary as array, total token count in corpus) */ private def preprocess( sc: SparkContext, paths: Seq[String], vocabSize: Int, - stopwordFile: String): (RDD[(Long, Vector)], Array[String]) = { + stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { // Get dataset of document texts // One document per line in each text file. @@ -185,23 +186,23 @@ object LDAExample { } tokenized.cache() - // Choose vocabulary: Map[word -> id] - val vocab: Map[String, Int] = { - // Counts words: RDD[(word, wordCount)] - val wordCounts: RDD[(String, Int)] = tokenized - .flatMap { case (_, tokens) => tokens.map(_ -> 1) } - .reduceByKey(_ + _) - val sortedWC = wordCounts - .sortBy(_._2, ascending = false) - val selectedWC = if (vocabSize == -1) { - sortedWC.collect() - } else { - sortedWC.take(vocabSize) - } + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Long)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1L) } + .reduceByKey(_ + _) + // Sort words, and select vocab + val sortedWC = wordCounts.sortBy(_._2, ascending = false) + val selectedWC: Array[(String, Long)] = if (vocabSize == -1) { + sortedWC.collect() + } else { + sortedWC.take(vocabSize) + } + val totalTokenCount = selectedWC.map(_._2).sum + // vocabulary: Map[word -> id] + val vocab: Map[String, Int] = selectedWC.map(_._1) .zipWithIndex .toMap - } val documents = tokenized.map { case (id, tokens) => // Filter tokens by vocabulary, and create word count vector representation of document. @@ -222,7 +223,7 @@ object LDAExample { val vocabArray = new Array[String](vocab.size) vocab.foreach { case (term, i) => vocabArray(i) = term } - (documents, vocabArray) + (documents, vocabArray, totalTokenCount) } } @@ -263,12 +264,20 @@ private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Se // Remove short words and strings that aren't only letters word match { case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => - words += word + words += w case _ => } current = end - end = wb.next() + try { + end = wb.next() + } catch { + case e: Exception => + // Ignore remaining text in line. + // This is a known bug in BreakIterator (for some Java versions), + // which fails when it sees certain characters. + end = BreakIterator.DONE + } } words } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala index 445a8603215cc..e3e0b21f23525 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala @@ -129,12 +129,11 @@ object LDATiming { for (vocabSize <- params.vocabSizes) { val preprocessStart = System.nanoTime() - val (corpus, vocabArray) = preprocess(sc, params.input, corpusSize, vocabSize, - params.stopwordFile) + val (corpus, vocabArray, actualNumTokens) = + preprocess(sc, params.input, corpusSize, vocabSize, params.stopwordFile) corpus.repartition(params.numPartitions).cache() val actualCorpusSize = corpus.count() val actualVocabSize = vocabArray.size - val actualNumTokens = corpus.map(_._2.toArray.sum.toLong).sum().toLong val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 println() @@ -199,13 +198,14 @@ object LDATiming { /** * Load documents, tokenize them, create vocabulary, and prepare documents as word count vectors. + * @return (corpus, vocabulary as array, total token count in corpus) */ private def preprocess( sc: SparkContext, paths: Seq[String], corpusSize: Long, vocabSize: Int, - stopwordFile: String): (RDD[(Long, Vector)], Array[String]) = { + stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { // Get dataset of document texts // One document per line in each text file. @@ -230,23 +230,23 @@ object LDATiming { } tokenized.cache() - // Choose vocabulary: Map[word -> id] - val vocab: Map[String, Int] = { - // Counts words: RDD[(word, wordCount)] - val wordCounts: RDD[(String, Int)] = tokenized - .flatMap { case (_, tokens) => tokens.map(_ -> 1) } - .reduceByKey(_ + _) - val sortedWC = wordCounts - .sortBy(_._2, ascending = false) - val selectedWC = if (vocabSize == -1) { - sortedWC.collect() - } else { - sortedWC.take(vocabSize) - } + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Long)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1L) } + .reduceByKey(_ + _) + // Sort words, and select vocab + val sortedWC = wordCounts.sortBy(_._2, ascending = false) + val selectedWC: Array[(String, Long)] = if (vocabSize == -1) { + sortedWC.collect() + } else { + sortedWC.take(vocabSize) + } + val totalTokenCount = selectedWC.map(_._2).sum + // vocabulary: Map[word -> id] + val vocab: Map[String, Int] = selectedWC.map(_._1) .zipWithIndex .toMap - } val documents = tokenized.map { case (id, tokens) => // Filter tokens by vocabulary, and create word count vector representation of document. @@ -267,7 +267,7 @@ object LDATiming { val vocabArray = new Array[String](vocab.size) vocab.foreach { case (term, i) => vocabArray(i) = term } - (documents, vocabArray) + (documents, vocabArray, totalTokenCount) } } @@ -302,12 +302,20 @@ class Tokenizer(sc: SparkContext, stopwordFile: String) extends Serializable { // Remove short words and strings that aren't only letters word match { case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => - words += word + words += w case _ => } current = end - end = wb.next() + try { + end = wb.next() + } catch { + case e: Exception => + // Ignore remaining text in line. + // This is a known bug in BreakIterator (for some Java versions), + // which fails when it sees certain characters. + end = BreakIterator.DONE + } } words } From e3914748187283a0ea3c929e91a0d5cae44e7b28 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 17:01:25 -0800 Subject: [PATCH 25/30] Removed LDATiming. Added PeriodicGraphCheckpointerSuite.scala. Small LDA cleanups. --- .../spark/examples/mllib/LDATiming.scala | 323 ------------------ .../apache/spark/mllib/clustering/LDA.scala | 4 +- .../spark/mllib/clustering/LDAModel.scala | 4 - .../impl/PeriodicGraphCheckpointer.scala | 39 ++- .../impl/PeriodicGraphCheckpointerSuite.scala | 182 ++++++++++ 5 files changed, 211 insertions(+), 341 deletions(-) delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala deleted file mode 100644 index e3e0b21f23525..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDATiming.scala +++ /dev/null @@ -1,323 +0,0 @@ -/* - * 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.examples.mllib - -import scala.collection.mutable - -import java.text.BreakIterator - -import scopt.OptionParser - -import org.apache.log4j.{Level, Logger} - -import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.mllib.clustering.LDA -import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.rdd.RDD - - -/** - * An app for timing Latent Dirichlet Allocation (LDA). - * - * This takes vectors of parameters: - * - corpusSize - * - vocabSize - * - k - * For each combination of values, it runs LDA and prints the time for iterations, - * the topics estimated, etc. - * - * Run with - * {{{ - * ./bin/run-example mllib.LDATiming [options] - * }}} - * If you use it as a template to create your own app, please use `spark-submit` to submit your app. - */ -object LDATiming { - - private case class Params( - input: Seq[String] = Seq.empty, - corpusSizes: Array[Long] = Array(-1), - ks: Array[Int] = Array(20), - numPartitions: Int = 16, - maxIterations: Int = 10, - docConcentration: Double = -1, - topicConcentration: Double = -1, - vocabSizes: Array[Int] = Array(10000), - stopwordFile: String = "", - checkpointDir: Option[String] = None, - checkpointInterval: Int = 10) extends AbstractParams[Params] - - def main(args: Array[String]) { - val defaultParams = Params() - - val parser = new OptionParser[Params]("LDATiming") { - head("LDATiming: an example LDA timing app for plain text data.") - opt[String]("corpusSizes") - .text(s"numbers of documents to test. default: ${defaultParams.corpusSizes}") - .action((x, c) => c.copy(corpusSizes = x.split("\\s").map(_.toLong))) - opt[String]("ks") - .text(s"numbers of topics to test. default: ${defaultParams.ks}") - .action((x, c) => c.copy(ks = x.split("\\s").map(_.toInt))) - opt[Int]("maxIterations") - .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}") - .action((x, c) => c.copy(maxIterations = x)) - opt[Int]("numPartitions") - .text(s"number of partitions. default: ${defaultParams.numPartitions}") - .action((x, c) => c.copy(numPartitions = x)) - opt[Double]("docConcentration") - .text(s"amount of topic smoothing to use. default: ${defaultParams.docConcentration}") - .action((x, c) => c.copy(docConcentration = x)) - opt[Double]("topicConcentration") - .text(s"amount of word smoothing to use. default: ${defaultParams.topicConcentration}") - .action((x, c) => c.copy(topicConcentration = x)) - opt[String]("vocabSizes") - .text(s"numbers of distinct word types to use, chosen by frequency." + - s" default: ${defaultParams.vocabSizes}") - .action((x, c) => c.copy(vocabSizes = x.split("\\s").map(_.toInt))) - opt[String]("stopwordFile") - .text(s"filepath for a list of stopwords. Note: This must fit on a single machine." + - s" default: ${defaultParams.stopwordFile}") - .action((x, c) => c.copy(stopwordFile = x)) - opt[String]("checkpointDir") - .text(s"Directory for checkpointing intermediate results." + - s" Checkpointing helps with recovery and eliminates temporary shuffle files on disk." + - s" default: ${defaultParams.checkpointDir}") - .action((x, c) => c.copy(checkpointDir = Some(x))) - opt[Int]("checkpointInterval") - .text(s"Iterations between each checkpoint. Only used if checkpointDir is set." + - s" default: ${defaultParams.checkpointInterval}") - .action((x, c) => c.copy(checkpointInterval = x)) - arg[String]("...") - .text("input paths (directories) to plain text corpora." - + " Each text file line should hold 1 document.") - .unbounded() - .required() - .action((x, c) => c.copy(input = c.input :+ x)) - } - - parser.parse(args, defaultParams).map { params => - run(params) - }.getOrElse { - parser.showUsageAsError - sys.exit(1) - } - } - - private def run(params: Params) { - val conf = new SparkConf().setAppName(s"LDATiming with $params") - val sc = new SparkContext(conf) - - Logger.getRootLogger.setLevel(Level.WARN) - - // Load documents, and prepare them for LDA. - for (corpusSize <- params.corpusSizes) { - for (vocabSize <- params.vocabSizes) { - - val preprocessStart = System.nanoTime() - val (corpus, vocabArray, actualNumTokens) = - preprocess(sc, params.input, corpusSize, vocabSize, params.stopwordFile) - corpus.repartition(params.numPartitions).cache() - val actualCorpusSize = corpus.count() - val actualVocabSize = vocabArray.size - val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 - - println() - println(s"DATASET with corpusSize=$corpusSize, vocabSize=$vocabSize:") - println(s"\t Training set size: $actualCorpusSize documents") - println(s"\t Vocabulary size: $actualVocabSize terms") - println(s"\t Training set size: $actualNumTokens tokens") - println(s"\t Preprocessing time: $preprocessElapsed sec") - println() - - for (k <- params.ks) { - // Run LDA. - val lda = new LDA() - lda.setK(k) - .setMaxIterations(params.maxIterations) - .setDocConcentration(params.docConcentration) - .setTopicConcentration(params.topicConcentration) - .setCheckpointInterval(params.checkpointInterval) - if (params.checkpointDir.nonEmpty) { - lda.setCheckpointDir(params.checkpointDir.get) - } - val startTime = System.nanoTime() - val ldaModel = lda.run(corpus) - val elapsed = (System.nanoTime() - startTime) / 1e9 - - println(s"Finished training LDA model. Summary:") - println(s"\t Training time: $elapsed sec") - val avgLogLikelihood = ldaModel.logLikelihood / actualCorpusSize.toDouble - println(s"\t Training data average log likelihood: $avgLogLikelihood") - println(s"\t Training times per iteration (sec):\n" + - s"${ldaModel.iterationTimes.mkString("\t", "\n\t", "\n")}") - println() - - // Print the topics, showing the top-weighted terms for each topic. - val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) - val topics = topicIndices.map { case (terms, termWeights) => - terms.zip(termWeights).map { case (term, weight) => (vocabArray(term.toInt), weight) } - } - println(s"$k topics:") - topics.zipWithIndex.foreach { case (topic, i) => - println(s"TOPIC $i") - topic.foreach { case (term, weight) => - println(s"$term\t$weight") - } - println() - } - println() - println("--------------------------------------------------------------") - println() - println(s"RESULTS: $corpusSize $vocabSize $actualCorpusSize $actualVocabSize" + - s" $actualNumTokens $k $elapsed $avgLogLikelihood" + - s" ${ldaModel.iterationTimes.mkString(" ")}") - println() - println("==============================================================") - println() - } - } - } - - sc.stop() - } - - /** - * Load documents, tokenize them, create vocabulary, and prepare documents as word count vectors. - * @return (corpus, vocabulary as array, total token count in corpus) - */ - private def preprocess( - sc: SparkContext, - paths: Seq[String], - corpusSize: Long, - vocabSize: Int, - stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { - - // Get dataset of document texts - // One document per line in each text file. - val files: Seq[RDD[String]] = for (p <- paths) yield { - sc.textFile(p) - } - val textRDD_tmp: RDD[String] = files.reduce(_ ++ _) // combine results from multiple paths - val origSize = textRDD_tmp.count() - - // Subsample data. - val textRDD: RDD[String] = if (corpusSize == -1 || corpusSize >= origSize) { - textRDD_tmp - } else { - textRDD_tmp.sample(withReplacement = true, fraction = corpusSize.toDouble / origSize, - seed = 123456) - } - - // Split text into words - val tokenizer = new Tokenizer(sc, stopwordFile) - val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => - id -> tokenizer.getWords(text) - } - tokenized.cache() - - // Counts words: RDD[(word, wordCount)] - val wordCounts: RDD[(String, Long)] = tokenized - .flatMap { case (_, tokens) => tokens.map(_ -> 1L) } - .reduceByKey(_ + _) - // Sort words, and select vocab - val sortedWC = wordCounts.sortBy(_._2, ascending = false) - val selectedWC: Array[(String, Long)] = if (vocabSize == -1) { - sortedWC.collect() - } else { - sortedWC.take(vocabSize) - } - val totalTokenCount = selectedWC.map(_._2).sum - // vocabulary: Map[word -> id] - val vocab: Map[String, Int] = - selectedWC.map(_._1) - .zipWithIndex - .toMap - - val documents = tokenized.map { case (id, tokens) => - // Filter tokens by vocabulary, and create word count vector representation of document. - val wc = new mutable.HashMap[Int, Int]() - tokens.foreach { term => - if (vocab.contains(term)) { - val termIndex = vocab(term) - wc(termIndex) = wc.getOrElse(termIndex, 0) + 1 - } - } - val indices = wc.keys.toArray.sorted - val values = indices.map(i => wc(i).toDouble) - - val sb = Vectors.sparse(vocab.size, indices, values) - (id, sb) - } - - val vocabArray = new Array[String](vocab.size) - vocab.foreach { case (term, i) => vocabArray(i) = term } - - (documents, vocabArray, totalTokenCount) - } -} - -class Tokenizer(sc: SparkContext, stopwordFile: String) extends Serializable { - - private val stopwords: Set[String] = if (stopwordFile.isEmpty) { - Set.empty[String] - } else { - val stopwordText = sc.textFile(stopwordFile).collect() - stopwordText.flatMap(_.stripMargin.split("\\s+")).toSet - } - - // Matches sequences of Unicode letters - private val allWordRegex = "^(\\p{L}*)$".r - - // Ignore words shorter than this length. - private val minWordLength = 3 - - def getWords(text: String): IndexedSeq[String] = { - val words = new mutable.ArrayBuffer[String]() - - // Use Java BreakIterator to tokenize text into words. - val wb = BreakIterator.getWordInstance - wb.setText(text) - - // current,end index start,end of each word - var current = wb.first() - var end = wb.next() - while (end != BreakIterator.DONE) { - // Convert to lowercase - val word: String = text.substring(current, end).toLowerCase - // Remove short words and strings that aren't only letters - word match { - case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => - words += w - case _ => - } - - current = end - try { - end = wb.next() - } catch { - case e: Exception => - // Ignore remaining text in line. - // This is a known bug in BreakIterator (for some Java versions), - // which fails when it sees certain characters. - end = BreakIterator.DONE - } - } - words - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 1755b995d57e9..f0bf2baf48e88 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -66,8 +66,6 @@ class LDA private ( private var checkpointDir: Option[String], private var checkpointInterval: Int) extends Logging { - import LDA._ - def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) @@ -311,7 +309,7 @@ private[clustering] object LDA { def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 /** - * State for EM algorithm: data + parameter graph, plus algorithm parameters. + * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. * * @param graph EM graph, storing current parameter estimates in vertex descriptors and * data (token counts) in edge descriptors. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index aade713753527..e82b0310c8ecc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -36,8 +36,6 @@ import org.apache.spark.util.BoundedPriorityQueue @DeveloperApi abstract class LDAModel private[clustering] { - import LDA._ - /** Number of topics */ def k: Int @@ -164,8 +162,6 @@ abstract class LDAModel private[clustering] { class LocalLDAModel private[clustering] ( private val topics: Matrix) extends LDAModel with Serializable { - import LDA._ - override def k: Int = topics.numCols override def vocabSize: Int = topics.numRows diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala index 864829e03849f..7631390e0e23e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -34,10 +34,35 @@ import org.apache.spark.storage.StorageLevel * checkpoint files are removed. * * Users should call [[PeriodicGraphCheckpointer.updateGraph()]] when a new graph has been created, - * before the graph has been materialized. + * before the graph has been materialized. When called, this does the following: + * - Persist new graph (if not yet persisted), and put in queue of persisted graphs. + * - Unpersist graphs from queue until there are at most 3 persisted graphs. + * - If using checkpointing and the checkpoint interval has been reached, + * - Checkpoint the new graph, and put in a queue of checkpointed graphs. + * - Remove older checkpoints. * - * NOTE: This class should NOT be copied (since copies may conflict on which Graphs should be - * checkpointed). + * WARNINGS: + * - This class should NOT be copied (since copies may conflict on which Graphs should be + * checkpointed). + * - This class removes checkpoint files once later graphs have been checkpointed. + * However, references to the older graphs will still return isCheckpointed = true. + * + * Example usage: + * val (graph1, graph2, graph3, ...) = ... + * val cp = new PeriodicGraphCheckpointer(graph, dir, 2) + * // persisted: graph1 + * cp.updateGraph(graph2) + * // persisted: graph1, graph2 + * // checkpointed: graph2 + * cp.updateGraph(graph3) + * // persisted: graph1, graph2, graph3 + * // checkpointed: graph2 + * cp.updateGraph(graph4) + * // persisted: graph2, graph3, graph4 + * // checkpointed: graph4 + * cp.updateGraph(graph5) + * // persisted: graph3, graph4, graph5 + * // checkpointed: graph4 * * @param currentGraph Initial graph * @param checkpointDir The directory for storing checkpoint files @@ -82,7 +107,6 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( */ def updateGraph(newGraph: Graph[VD, ED]): Unit = { if (newGraph.vertices.getStorageLevel == StorageLevel.NONE) { - println(s"PeriodicGraphCheckpointer.updateGraph: persisting ${newGraph.vertices.id}") newGraph.persist() } persistedQueue.enqueue(newGraph) @@ -91,8 +115,6 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( // before the graph has been materialized. while (persistedQueue.size > 3) { val graphToUnpersist = persistedQueue.dequeue() - println(s"PeriodicGraphCheckpointer.updateGraph:" + - s" unpersisting ${graphToUnpersist.vertices.id}") graphToUnpersist.unpersist(blocking = false) } updateCount += 1 @@ -100,7 +122,6 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( // Handle checkpointing (after persisting) if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) { // Add new checkpoint before removing old checkpoints. - println(s"PeriodicGraphCheckpointer.updateGraph: checkpointing ${newGraph.vertices.id}") newGraph.checkpoint() checkpointQueue.enqueue(newGraph) // Remove checkpoints before the latest one. @@ -131,17 +152,13 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( */ private def removeCheckpointFile(): Unit = { val old = checkpointQueue.dequeue() - println(s"PeriodicGraphCheckpointer.updateGraph: removing checkpoint ${old.vertices.id}") // Since the old checkpoint is not deleted by Spark, we manually delete it. val fs = FileSystem.get(sc.hadoopConfiguration) old.getCheckpointFiles.foreach { checkpointFile => try { - println(s" --removing file: $checkpointFile") fs.delete(new Path(checkpointFile), true) } catch { case e: Exception => - println("PeriodicGraphCheckpointer could not remove old checkpoint file: " + - checkpointFile) logWarning("PeriodicGraphCheckpointer could not remove old checkpoint file: " + checkpointFile) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala new file mode 100644 index 0000000000000..4ca38f2b8c35e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala @@ -0,0 +1,182 @@ +/* + * 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.mllib.impl + +import org.scalatest.FunSuite + +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.SparkContext +import org.apache.spark.graphx.{Edge, Graph} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + + +class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext { + + import PeriodicGraphCheckpointerSuite._ + + // TODO: Do I need to call count() on the graphs' RDDs? + + test("Persisting") { + var graphsToCheck = Seq.empty[GraphToCheck] + + val graph1 = createGraph(sc) + val checkpointer = new PeriodicGraphCheckpointer(graph1, None, 10) + graph1.vertices.count() + graph1.edges.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkPersistence(graphsToCheck, 1) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.updateGraph(graph) + graph.vertices.count() + graph.edges.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkPersistence(graphsToCheck, iteration) + iteration += 1 + } + } + + test("Checkpointing") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val checkpointInterval = 2 + var graphsToCheck = Seq.empty[GraphToCheck] + + val graph1 = createGraph(sc) + val checkpointer = new PeriodicGraphCheckpointer(graph1, Some(path), checkpointInterval) + graph1.edges.map(_.attr).count() + graph1.vertices.map(_._2).count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkCheckpoint(graphsToCheck, 1, checkpointInterval) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.updateGraph(graph) + graph.vertices.map(_._2).count() + graph.edges.map(_.attr).count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkCheckpoint(graphsToCheck, iteration, checkpointInterval) + iteration += 1 + } + + Utils.deleteRecursively(tempDir) + } +} + +private object PeriodicGraphCheckpointerSuite { + + case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) + + val edges = Seq( + Edge[Double](0, 1, 0), + Edge[Double](1, 2, 0), + Edge[Double](2, 3, 0), + Edge[Double](3, 4, 0)) + + def createGraph(sc: SparkContext): Graph[Double, Double] = { + Graph.fromEdges[Double, Double](sc.parallelize(edges), 0) + } + + def checkPersistence(graphs: Seq[GraphToCheck], iteration: Int): Unit = { + graphs.foreach { g => + checkPersistence(g.graph, g.gIndex, iteration) + } + } + + /** + * Check storage level of graph. + * @param gIndex Index of graph in order inserted into checkpointer (from 1). + * @param iteration Total number of graphs inserted into checkpointer. + */ + def checkPersistence(graph: Graph[_, _], gIndex: Int, iteration: Int): Unit = { + try { + if (gIndex + 2 < iteration) { + assert(graph.vertices.getStorageLevel == StorageLevel.NONE) + assert(graph.edges.getStorageLevel == StorageLevel.NONE) + } else { + assert(graph.vertices.getStorageLevel != StorageLevel.NONE) + assert(graph.edges.getStorageLevel != StorageLevel.NONE) + } + } catch { + case _: AssertionError => + throw new Exception(s"PeriodicGraphCheckpointerSuite.checkPersistence failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t graph.vertices.getStorageLevel = ${graph.vertices.getStorageLevel}\n" + + s"\t graph.edges.getStorageLevel = ${graph.edges.getStorageLevel}\n") + } + } + + def checkCheckpoint(graphs: Seq[GraphToCheck], iteration: Int, checkpointInterval: Int): Unit = { + graphs.reverse.foreach { g => + checkCheckpoint(g.graph, g.gIndex, iteration, checkpointInterval) + } + } + + /** + * Check checkpointed status of graph. + * @param gIndex Index of graph in order inserted into checkpointer (from 1). + * @param iteration Total number of graphs inserted into checkpointer. + */ + def checkCheckpoint( + graph: Graph[_, _], + gIndex: Int, + iteration: Int, + checkpointInterval: Int): Unit = { + try { + if (gIndex % checkpointInterval == 0) { + // We allow 2 checkpoint intervals since we perform an action (checkpointing a second graph) + // only AFTER PeriodicGraphCheckpointer decides whether to remove the previous checkpoint. + if (iteration - 2 * checkpointInterval < gIndex && gIndex <= iteration) { + assert(graph.isCheckpointed, "Graph should be checkpointed") + assert(graph.getCheckpointFiles.length == 2, "Graph should have 2 checkpoint files") + } else { + // Note: We cannot check graph.isCheckpointed since that value is never updated. + // Instead, we check for the presence of the checkpoint files. + // This test should continue to work even after this graph.isCheckpointed issue + // is fixed (though it can then be simplified and not look for the files). + val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration) + graph.getCheckpointFiles.foreach { checkpointFile => + assert(!fs.exists(new Path(checkpointFile)), + "Graph checkpoint file should have been removed") + } + } + } else { + // Graph should never be checkpointed + assert(!graph.isCheckpointed, "Graph should never have been checkpointed") + assert(graph.getCheckpointFiles.length == 0, "Graph should not have any checkpoint files") + } + } catch { + case e: AssertionError => + throw new Exception(s"PeriodicGraphCheckpointerSuite.checkCheckpoint failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t checkpointInterval = $checkpointInterval\n" + + s"\t graph.isCheckpointed = ${graph.isCheckpointed}\n" + + s"\t graph.getCheckpointFiles = ${graph.getCheckpointFiles.mkString(", ")}\n" + + s" AssertionError message: ${e.getMessage}") + } + } + +} From 4ae2a7df685badae3b7a3b8e94a871186c148a78 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 17:02:49 -0800 Subject: [PATCH 26/30] removed duplicate graphx dependency in mllib/pom.xml --- mllib/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index f1f8c886f949e..a8cee3d51a780 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -55,11 +55,6 @@ spark-graphx_${scala.binary.version} ${project.version} - - org.apache.spark - spark-graphx_${scala.binary.version} - ${project.version} - org.jblas jblas From e3980d2967ae45d1477384a055abd1c9d7975dbd Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 17:15:02 -0800 Subject: [PATCH 27/30] cleaned up PeriodicGraphCheckpointerSuite.scala --- .../mllib/impl/PeriodicGraphCheckpointerSuite.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala index 4ca38f2b8c35e..35622fef9cb0a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala @@ -39,8 +39,6 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext val graph1 = createGraph(sc) val checkpointer = new PeriodicGraphCheckpointer(graph1, None, 10) - graph1.vertices.count() - graph1.edges.count() graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) checkPersistence(graphsToCheck, 1) @@ -48,8 +46,6 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext while (iteration < 9) { val graph = createGraph(sc) checkpointer.updateGraph(graph) - graph.vertices.count() - graph.edges.count() graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) checkPersistence(graphsToCheck, iteration) iteration += 1 @@ -64,8 +60,8 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext val graph1 = createGraph(sc) val checkpointer = new PeriodicGraphCheckpointer(graph1, Some(path), checkpointInterval) - graph1.edges.map(_.attr).count() - graph1.vertices.map(_._2).count() + graph1.edges.count() + graph1.vertices.count() graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) checkCheckpoint(graphsToCheck, 1, checkpointInterval) @@ -73,8 +69,8 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext while (iteration < 9) { val graph = createGraph(sc) checkpointer.updateGraph(graph) - graph.vertices.map(_._2).count() - graph.edges.map(_.attr).count() + graph.vertices.count() + graph.edges.count() graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) checkCheckpoint(graphsToCheck, iteration, checkpointInterval) iteration += 1 From 589728b436b9173c880654fe97a5a9640aee639b Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 20:57:55 -0800 Subject: [PATCH 28/30] Updates per code review. Main change was in LDAExample for faster vocab computation. Also updated PeriodicGraphCheckpointerSuite.scala to clean up checkpoint files at end --- .../spark/examples/mllib/LDAExample.scala | 36 +++++----- .../apache/spark/mllib/clustering/LDA.scala | 71 ++++++++++--------- .../spark/mllib/clustering/LDAModel.scala | 40 +++++------ .../impl/PeriodicGraphCheckpointer.scala | 4 ++ .../spark/mllib/clustering/JavaLDASuite.java | 11 +-- .../impl/PeriodicGraphCheckpointerSuite.scala | 27 ++++--- 6 files changed, 99 insertions(+), 90 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 5d65b0f33c1fe..f4c545ad70e96 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -17,10 +17,10 @@ package org.apache.spark.examples.mllib -import scala.collection.mutable - import java.text.BreakIterator +import scala.collection.mutable + import scopt.OptionParser import org.apache.log4j.{Level, Logger} @@ -174,10 +174,7 @@ object LDAExample { // Get dataset of document texts // One document per line in each text file. - val files: Seq[RDD[String]] = for (p <- paths) yield { - sc.textFile(p) - } - val textRDD: RDD[String] = files.reduce(_ ++ _) // combine results from multiple paths + val textRDD: RDD[String] = sc.textFile(paths.mkString(",")) // Split text into words val tokenizer = new SimpleTokenizer(sc, stopwordFile) @@ -190,19 +187,20 @@ object LDAExample { val wordCounts: RDD[(String, Long)] = tokenized .flatMap { case (_, tokens) => tokens.map(_ -> 1L) } .reduceByKey(_ + _) - // Sort words, and select vocab - val sortedWC = wordCounts.sortBy(_._2, ascending = false) - val selectedWC: Array[(String, Long)] = if (vocabSize == -1) { - sortedWC.collect() - } else { - sortedWC.take(vocabSize) + wordCounts.cache() + val fullVocabSize = wordCounts.count() + // Select vocab + // (vocab: Map[word -> id], total tokens after selecting vocab) + val (vocab: Map[String, Int], selectedTokenCount: Long) = { + val tmpSortedWC: Array[(String, Long)] = if (vocabSize == -1 || fullVocabSize <= vocabSize) { + // Use all terms + wordCounts.collect().sortBy(-_._2) + } else { + // Sort terms to select vocab + wordCounts.sortBy(_._2, ascending = false).take(vocabSize) + } + (tmpSortedWC.map(_._1).zipWithIndex.toMap, tmpSortedWC.map(_._2).sum) } - val totalTokenCount = selectedWC.map(_._2).sum - // vocabulary: Map[word -> id] - val vocab: Map[String, Int] = - selectedWC.map(_._1) - .zipWithIndex - .toMap val documents = tokenized.map { case (id, tokens) => // Filter tokens by vocabulary, and create word count vector representation of document. @@ -223,7 +221,7 @@ object LDAExample { val vocabArray = new Array[String](vocab.size) vocab.foreach { case (term, i) => vocabArray(i) = term } - (documents, vocabArray, totalTokenCount) + (documents, vocabArray, selectedTokenCount) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index f0bf2baf48e88..d8f39d1287bab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -22,8 +22,8 @@ import java.util.Random import breeze.linalg.{DenseVector => BDV, normalize, axpy => brzAxpy} import org.apache.spark.Logging -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.api.java.JavaRDD +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer @@ -33,7 +33,7 @@ import org.apache.spark.util.Utils /** - * :: DeveloperApi :: + * :: Experimental :: * * Latent Dirichlet Allocation (LDA), a topic model designed for text documents. * @@ -52,11 +52,8 @@ import org.apache.spark.util.Utils * - Paper which clearly explains several algorithms, including EM: * Asuncion, Welling, Smyth, and Teh. * "On Smoothing and Inference for Topic Models." UAI, 2009. - * - * NOTE: This is currently marked DeveloperApi since it is under active development and may undergo - * API changes. */ -@DeveloperApi +@Experimental class LDA private ( private var k: Int, private var maxIterations: Int, @@ -69,18 +66,26 @@ class LDA private ( def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) + def getK: Int = k + /** * Number of topics to infer. I.e., the number of soft cluster centers. * (default = 10) */ - def getK: Int = k - def setK(k: Int): this.type = { require(k > 0, s"LDA k (number of clusters) must be > 0, but was set to $k") this.k = k this } + def getDocConcentration: Double = { + if (this.docConcentration == -1) { + (50.0 / k) + 1.0 + } else { + this.docConcentration + } + } + /** * Topic smoothing parameter (commonly named "alpha"). * @@ -99,14 +104,6 @@ class LDA private ( * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), * but values in (0,1) are not yet supported. */ - def getDocConcentration: Double = { - if (this.docConcentration == -1) { - (50.0 / k) + 1.0 - } else { - this.docConcentration - } - } - def setDocConcentration(docConcentration: Double): this.type = { require(docConcentration > 1.0 || docConcentration == -1.0, s"LDA docConcentration must be > 1.0 (or -1 for auto), but was set to $docConcentration") @@ -120,6 +117,14 @@ class LDA private ( /** Alias for [[setDocConcentration()]] */ def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) + def getTopicConcentration: Double = { + if (this.topicConcentration == -1) { + 1.1 + } else { + this.topicConcentration + } + } + /** * Term smoothing parameter (commonly named "beta" or "eta"). * @@ -139,14 +144,6 @@ class LDA private ( * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), * but values in (0,1) are not yet supported. */ - def getTopicConcentration: Double = { - if (this.topicConcentration == -1) { - 1.1 - } else { - this.topicConcentration - } - } - def setTopicConcentration(topicConcentration: Double): this.type = { require(topicConcentration > 1.0 || topicConcentration == -1.0, s"LDA topicConcentration must be > 1.0 (or -1 for auto), but was set to $topicConcentration") @@ -160,33 +157,38 @@ class LDA private ( /** Alias for [[setTopicConcentration()]] */ def setBeta(beta: Double): this.type = setBeta(beta) + def getMaxIterations: Int = maxIterations + /** * Maximum number of iterations for learning. * (default = 20) */ - def getMaxIterations: Int = maxIterations - def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this } - /** Random seed */ def getSeed: Long = seed + /** Random seed */ def setSeed(seed: Long): this.type = { this.seed = seed this } + def getCheckpointDir: Option[String] = checkpointDir + /** * Directory for storing checkpoint files during learning. * This is not necessary, but checkpointing helps with recovery (when nodes fail). * It also helps with eliminating temporary shuffle files on disk, which can be important when * LDA is run for many iterations. + * + * NOTE: If the [[org.apache.spark.SparkContext.checkpointDir]] is already set, then the value + * given to LDA is ignored, and the existing directory is kept. + * + * (default = None) */ - def getCheckpointDir: Option[String] = checkpointDir - def setCheckpointDir(checkpointDir: String): this.type = { this.checkpointDir = Some(checkpointDir) this @@ -197,12 +199,13 @@ class LDA private ( this } + def getCheckpointInterval: Int = checkpointInterval + /** * Period (in iterations) between checkpoints. + * (default = 10) * @see [[getCheckpointDir]] */ - def getCheckpointInterval: Int = checkpointInterval - def setCheckpointInterval(checkpointInterval: Int): this.type = { this.checkpointInterval = checkpointInterval this @@ -234,8 +237,8 @@ class LDA private ( } /** Java-friendly version of [[run()]] */ - def run(documents: JavaRDD[(java.lang.Long, Vector)]): DistributedLDAModel = { - run(documents.rdd.map(id_counts => (id_counts._1.asInstanceOf[Long], id_counts._2))) + def run(documents: JavaPairRDD[java.lang.Long, Vector]): DistributedLDAModel = { + run(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index e82b0310c8ecc..19e8aab6eabd7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -19,21 +19,21 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseMatrix => BDM, normalize, sum => brzSum} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.Experimental import org.apache.spark.graphx.{VertexId, EdgeContext, Graph} import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} import org.apache.spark.rdd.RDD import org.apache.spark.util.BoundedPriorityQueue /** - * :: DeveloperApi :: + * :: Experimental :: * * Latent Dirichlet Allocation (LDA) model. * * This abstraction permits for different underlying representations, * including local and distributed data structures. */ -@DeveloperApi +@Experimental abstract class LDAModel private[clustering] { /** Number of topics */ @@ -57,8 +57,8 @@ abstract class LDAModel private[clustering] { * To get a more precise set of top terms, increase maxTermsPerTopic. * * @param maxTermsPerTopic Maximum number of terms to collect for each topic. - * @return Array over topics, where each element is a set of top terms represented - * as (term index, term weight in topic). + * @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. */ def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] @@ -68,8 +68,8 @@ abstract class LDAModel private[clustering] { * * WARNING: If vocabSize and k are large, this can return a large object! * - * @return Array over topics, where each element is a set of top terms represented - * as (term index, term weight in topic). + * @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. */ def describeTopics(): Array[(Array[Int], Array[Double])] = describeTopics(vocabSize) @@ -86,9 +86,9 @@ abstract class LDAModel private[clustering] { * To get a more precise set of top terms, increase maxTermsPerTopic. * * @param maxTermsPerTopic Maximum number of terms to collect for each topic. - * @return Array over topics, where each element is a set of top terms represented - * as (term, term weight in topic), where "term" is either the actual term text - * (if available) or the term index. + * @return Array over topics. Each topic is represented as a pair of matching arrays: + * (terms, term weights in topic) where terms are either the actual term text + * (if available) or the term indices. * Each topic's terms are sorted in order of decreasing weight. */ // def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[(Array[Double], Array[String])] @@ -102,9 +102,9 @@ abstract class LDAModel private[clustering] { * * WARNING: If vocabSize and k are large, this can return a large object! * - * @return Array over topics, where each element is a set of top terms represented - * as (term, term weight in topic), where "term" is either the actual term text - * (if available) or the term index. + * @return Array over topics. Each topic is represented as a pair of matching arrays: + * (terms, term weights in topic) where terms are either the actual term text + * (if available) or the term indices. * Each topic's terms are sorted in order of decreasing weight. */ // def describeTopicsAsStrings(): Array[(Array[Double], Array[String])] = @@ -146,19 +146,16 @@ abstract class LDAModel private[clustering] { } /** - * :: DeveloperApi :: + * :: Experimental :: * * Local LDA model. * This model stores only the inferred topics. * It may be used for computing topics for new documents, but it may give less accurate answers * than the [[DistributedLDAModel]]. * - * NOTE: This is currently marked DeveloperApi since it is under active development and may undergo - * API changes. - * * @param topics Inferred topics (vocabSize x k matrix). */ -@DeveloperApi +@Experimental class LocalLDAModel private[clustering] ( private val topics: Matrix) extends LDAModel with Serializable { @@ -187,17 +184,14 @@ class LocalLDAModel private[clustering] ( } /** - * :: DeveloperApi :: + * :: Experimental :: * * Distributed LDA model. * This model stores the inferred topics, the full training dataset, and the topic distributions. * When computing topics for new documents, it may give more accurate answers * than the [[LocalLDAModel]]. - * - * NOTE: This is currently marked DeveloperApi since it is under active development and may undergo - * API changes. */ -@DeveloperApi +@Experimental class DistributedLDAModel private ( private val graph: Graph[LDA.TopicCounts, LDA.TokenCount], private val globalTopicTotals: LDA.TopicCounts, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala index 7631390e0e23e..20ff0a2910ac5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -48,6 +48,7 @@ import org.apache.spark.storage.StorageLevel * However, references to the older graphs will still return isCheckpointed = true. * * Example usage: + * {{{ * val (graph1, graph2, graph3, ...) = ... * val cp = new PeriodicGraphCheckpointer(graph, dir, 2) * // persisted: graph1 @@ -63,12 +64,15 @@ import org.apache.spark.storage.StorageLevel * cp.updateGraph(graph5) * // persisted: graph3, graph4, graph5 * // checkpointed: graph4 + * }}} * * @param currentGraph Initial graph * @param checkpointDir The directory for storing checkpoint files * @param checkpointInterval Graphs will be checkpointed at this interval * @tparam VD Vertex descriptor type * @tparam ED Edge descriptor type + * + * TODO: Generalize this for Graphs and RDDs, and move it out of MLlib. */ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( var currentGraph: Graph[VD, ED], diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index 9fc561bb3b332..dc10aa67c7c1f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.ArrayList; +import org.apache.spark.api.java.JavaRDD; import scala.Tuple2; import org.junit.After; @@ -28,7 +29,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; @@ -40,12 +41,13 @@ public class JavaLDASuite implements Serializable { @Before public void setUp() { sc = new JavaSparkContext("local", "JavaLDA"); - tinyCorpus = new ArrayList>(); + ArrayList> tinyCorpus = new ArrayList>(); for (int i = 0; i < LDASuite$.MODULE$.tinyCorpus().length; i++) { tinyCorpus.add(new Tuple2((Long)LDASuite$.MODULE$.tinyCorpus()[i]._1(), LDASuite$.MODULE$.tinyCorpus()[i]._2())); } - corpus = sc.parallelize(tinyCorpus, 2); + JavaRDD> tmpCorpus = sc.parallelize(tinyCorpus, 2); + corpus = JavaPairRDD.fromJavaRDD(tmpCorpus); } @After @@ -112,7 +114,6 @@ public void distributedLDAModel() { private static Matrix tinyTopics = LDASuite$.MODULE$.tinyTopics(); private static Tuple2[] tinyTopicDescription = LDASuite$.MODULE$.tinyTopicDescription(); - private ArrayList> tinyCorpus; - JavaRDD> corpus; + JavaPairRDD corpus; } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala index 35622fef9cb0a..dac28a369b5b2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala @@ -76,6 +76,11 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext iteration += 1 } + checkpointer.deleteAllCheckpoints() + graphsToCheck.foreach { graph => + confirmCheckpointRemoved(graph.graph) + } + Utils.deleteRecursively(tempDir) } } @@ -130,6 +135,18 @@ private object PeriodicGraphCheckpointerSuite { } } + def confirmCheckpointRemoved(graph: Graph[_, _]): Unit = { + // Note: We cannot check graph.isCheckpointed since that value is never updated. + // Instead, we check for the presence of the checkpoint files. + // This test should continue to work even after this graph.isCheckpointed issue + // is fixed (though it can then be simplified and not look for the files). + val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration) + graph.getCheckpointFiles.foreach { checkpointFile => + assert(!fs.exists(new Path(checkpointFile)), + "Graph checkpoint file should have been removed") + } + } + /** * Check checkpointed status of graph. * @param gIndex Index of graph in order inserted into checkpointer (from 1). @@ -148,15 +165,7 @@ private object PeriodicGraphCheckpointerSuite { assert(graph.isCheckpointed, "Graph should be checkpointed") assert(graph.getCheckpointFiles.length == 2, "Graph should have 2 checkpoint files") } else { - // Note: We cannot check graph.isCheckpointed since that value is never updated. - // Instead, we check for the presence of the checkpoint files. - // This test should continue to work even after this graph.isCheckpointed issue - // is fixed (though it can then be simplified and not look for the files). - val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration) - graph.getCheckpointFiles.foreach { checkpointFile => - assert(!fs.exists(new Path(checkpointFile)), - "Graph checkpoint file should have been removed") - } + confirmCheckpointRemoved(graph) } } else { // Graph should never be checkpointed From 5c74345d303e1fff5f22c56e476daf7f0ea54995 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 22:42:53 -0800 Subject: [PATCH 29/30] cleaned up doc based on code review --- .../apache/spark/mllib/clustering/LDA.scala | 54 ++++++++++++++++--- .../impl/PeriodicGraphCheckpointer.scala | 19 ++++--- 2 files changed, 60 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index d8f39d1287bab..edeaace22d1d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -66,6 +66,9 @@ class LDA private ( def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) + /** + * Number of topics to infer. I.e., the number of soft cluster centers. + */ def getK: Int = k /** @@ -78,6 +81,12 @@ class LDA private ( this } + /** + * Concentration parameter (commonly named "alpha") for the prior placed on documents' + * distributions over topics ("theta"). + * + * This is the parameter to a symmetric Dirichlet distribution ("theta"). + */ def getDocConcentration: Double = { if (this.docConcentration == -1) { (50.0 / k) + 1.0 @@ -87,10 +96,10 @@ class LDA private ( } /** - * Topic smoothing parameter (commonly named "alpha"). + * Concentration parameter (commonly named "alpha") for the prior placed on documents' + * distributions over topics ("theta"). * - * This is the parameter to the Dirichlet prior placed on each document's distribution over topics - * ("theta"). We use a symmetric Dirichlet prior. + * This is the parameter to a symmetric Dirichlet distribution ("theta"). * * This value should be > 1.0, where larger values mean more smoothing (more regularization). * If set to -1, then docConcentration is set automatically. @@ -117,6 +126,15 @@ class LDA private ( /** Alias for [[setDocConcentration()]] */ def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) + /** + * 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 ("theta"). + * + * 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. + */ def getTopicConcentration: Double = { if (this.topicConcentration == -1) { 1.1 @@ -126,11 +144,13 @@ class LDA private ( } /** - * Term smoothing parameter (commonly named "beta" or "eta"). + * Concentration parameter (commonly named "beta" or "eta") for the prior placed on topics' + * distributions over terms. * - * This is the parameter to the Dirichlet prior placed on each topic's distribution over terms - * (which 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). + * This is the parameter to a symmetric Dirichlet distribution ("theta"). + * + * 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. * * This value should be > 0.0. * If set to -1, then topicConcentration is set automatically. @@ -157,6 +177,9 @@ class LDA private ( /** Alias for [[setTopicConcentration()]] */ def setBeta(beta: Double): this.type = setBeta(beta) + /** + * Maximum number of iterations for learning. + */ def getMaxIterations: Int = maxIterations /** @@ -168,6 +191,7 @@ class LDA private ( this } + /** Random seed */ def getSeed: Long = seed /** Random seed */ @@ -176,6 +200,15 @@ class LDA private ( this } + /** + * Directory for storing checkpoint files during learning. + * This is not necessary, but checkpointing helps with recovery (when nodes fail). + * It also helps with eliminating temporary shuffle files on disk, which can be important when + * LDA is run for many iterations. + * + * NOTE: If the [[org.apache.spark.SparkContext.checkpointDir]] is already set, then the value + * given to LDA is ignored, and the existing directory is kept. + */ def getCheckpointDir: Option[String] = checkpointDir /** @@ -194,11 +227,18 @@ class LDA private ( this } + /** + * Remove the directory for storing checkpoint files during learning. + */ def clearCheckpointDir(): this.type = { this.checkpointDir = None this } + /** + * Period (in iterations) between checkpoints. + * @see [[getCheckpointDir]] + */ def getCheckpointInterval: Int = checkpointInterval /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala index 20ff0a2910ac5..76672fe51e834 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -28,13 +28,15 @@ import org.apache.spark.storage.StorageLevel /** * This class helps with persisting and checkpointing Graphs. - * - * This class maintains a FIFO queue of Graphs, each of which is persisted and some of which are - * checkpointed. Once one Graph has been checkpointed, then previous RDDs are unpersisted and their - * checkpoint files are removed. + * Specifically, it automatically handles persisting and (optionally) checkpointing, as well as + * unpersisting and removing checkpoint files. * * Users should call [[PeriodicGraphCheckpointer.updateGraph()]] when a new graph has been created, - * before the graph has been materialized. When called, this does the following: + * before the graph has been materialized. After updating [[PeriodicGraphCheckpointer]], users are + * responsible for materializing the graph to ensure that persisting and checkpointing actually + * occur. + * + * When [[PeriodicGraphCheckpointer.updateGraph()]] is called, this does the following: * - Persist new graph (if not yet persisted), and put in queue of persisted graphs. * - Unpersist graphs from queue until there are at most 3 persisted graphs. * - If using checkpointing and the checkpoint interval has been reached, @@ -50,18 +52,23 @@ import org.apache.spark.storage.StorageLevel * Example usage: * {{{ * val (graph1, graph2, graph3, ...) = ... - * val cp = new PeriodicGraphCheckpointer(graph, dir, 2) + * val cp = new PeriodicGraphCheckpointer(graph1, dir, 2) + * graph1.vertices.count(); graph1.edges.count() * // persisted: graph1 * cp.updateGraph(graph2) + * graph2.vertices.count(); graph2.edges.count() * // persisted: graph1, graph2 * // checkpointed: graph2 * cp.updateGraph(graph3) + * graph3.vertices.count(); graph3.edges.count() * // persisted: graph1, graph2, graph3 * // checkpointed: graph2 * cp.updateGraph(graph4) + * graph4.vertices.count(); graph4.edges.count() * // persisted: graph2, graph3, graph4 * // checkpointed: graph4 * cp.updateGraph(graph5) + * graph5.vertices.count(); graph5.edges.count() * // persisted: graph3, graph4, graph5 * // checkpointed: graph4 * }}} From 77e8814073c10ee8f8753eb68412b1b86450af31 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 22:46:21 -0800 Subject: [PATCH 30/30] small doc fix --- .../org/apache/spark/mllib/clustering/LDA.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index edeaace22d1d6..d8f82867a09d2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -85,7 +85,7 @@ class LDA private ( * Concentration parameter (commonly named "alpha") for the prior placed on documents' * distributions over topics ("theta"). * - * This is the parameter to a symmetric Dirichlet distribution ("theta"). + * This is the parameter to a symmetric Dirichlet distribution. */ def getDocConcentration: Double = { if (this.docConcentration == -1) { @@ -99,7 +99,7 @@ class LDA private ( * Concentration parameter (commonly named "alpha") for the prior placed on documents' * distributions over topics ("theta"). * - * This is the parameter to a symmetric Dirichlet distribution ("theta"). + * This is the parameter to a symmetric Dirichlet distribution. * * This value should be > 1.0, where larger values mean more smoothing (more regularization). * If set to -1, then docConcentration is set automatically. @@ -130,7 +130,7 @@ class LDA private ( * 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 ("theta"). + * 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. @@ -147,7 +147,7 @@ class LDA private ( * 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 ("theta"). + * 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. @@ -205,9 +205,6 @@ class LDA private ( * This is not necessary, but checkpointing helps with recovery (when nodes fail). * It also helps with eliminating temporary shuffle files on disk, which can be important when * LDA is run for many iterations. - * - * NOTE: If the [[org.apache.spark.SparkContext.checkpointDir]] is already set, then the value - * given to LDA is ignored, and the existing directory is kept. */ def getCheckpointDir: Option[String] = checkpointDir @@ -228,7 +225,9 @@ class LDA private ( } /** - * Remove the directory for storing checkpoint files during learning. + * Clear the directory for storing checkpoint files during learning. + * If one is already set in the [[org.apache.spark.SparkContext]], then checkpointing will still + * occur; otherwise, no checkpointing will be used. */ def clearCheckpointDir(): this.type = { this.checkpointDir = None