From 107a0ba9605241653a85b113661a8fa5c055529f Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 4 Jun 2014 12:54:22 -0700 Subject: [PATCH 01/38] added Sebastian's CooccurrenceAnalysis patch updated it to use current Mahout-DSL --- .gitignore | 1 + spark/src/.DS_Store | Bin 0 -> 6148 bytes spark/src/main/.DS_Store | Bin 0 -> 6148 bytes spark/src/main/scala/.DS_Store | Bin 0 -> 6148 bytes spark/src/main/scala/org/.DS_Store | Bin 0 -> 6148 bytes spark/src/main/scala/org/apache/.DS_Store | Bin 0 -> 6148 bytes .../mahout/cf/CooccurrenceAnalysis.scala | 210 ++++++++++++++++++ .../mahout/cf/examples/Recommendations.scala | 169 ++++++++++++++ 8 files changed, 380 insertions(+) create mode 100644 spark/src/.DS_Store create mode 100644 spark/src/main/.DS_Store create mode 100644 spark/src/main/scala/.DS_Store create mode 100644 spark/src/main/scala/org/.DS_Store create mode 100644 spark/src/main/scala/org/apache/.DS_Store create mode 100644 spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala create mode 100644 spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala diff --git a/.gitignore b/.gitignore index c47bff17ce..f5003759e9 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ output-asf-email-examples/ .project .settings/ .idea/ +.DS_Store *.iml target/ examples/bin/tmp diff --git a/spark/src/.DS_Store b/spark/src/.DS_Store new file mode 100644 index 0000000000000000000000000000000000000000..7b0d36729e2ee777a660f9e8c6709dd97bc2fb68 GIT binary patch literal 6148 zcmeH~F>V4u3`M`g7D#EfOgRk)$PGpaPQV2S&>%r5BKkQx-ySzvsH0W%E!l7ES!;KH zv9ktX>-*^w7y&HlPOLmk%$N_j;tOYdpMH*)!|itQBJHgMp3+B5_H$d10#ZNMM4T9irw zDe%t}uwk>?toc%Twm!X{*Y{cVb)%DUIm54?049DEf6&9YUwlEvIYIqf#4$mI7itHYo8^+Vg;}!TM!kPMk^St>SKu2 zy&WuhT}`%Nw2S8Op?PPuDF&v|E?SVlv^p5502LT1&_&+c`M-yMoBu~GOsN1B_%j7` zzB}x;c&R*FKVHx3$E@1A!9l+q;q4~?i5fKY%m813XGbB*6r9*;8fO-L>F+DnJEpT><+(6u4ncTmt>mfx%k<;0j?k%)OTY z77GAt;u44mOoIvxs%DF!K}Wn~UQJvAgD#rQhvvHE{`aI^s?T@@K$wp;3YVR^S&eHWjb{ literal 0 HcmV?d00001 diff --git a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala new file mode 100644 index 0000000000..5df329b3be --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -0,0 +1,210 @@ +/* + * 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.mahout.cf + +import org.apache.mahout.math._ +import scalabindings._ +import RLikeOps._ +import drm._ +import RLikeDrmOps._ +import org.apache.mahout.sparkbindings._ + +import scala.collection.JavaConversions._ +import org.apache.mahout.math.stats.LogLikelihood +import collection._ +// import scala.collection.parallel.mutable +import org.apache.mahout.common.RandomUtils + + +/** + * based on "Ted Dunnning & Ellen Friedman: Practical Machine Learning, Innovations in Recommendation", + * available at http://www.mapr.com/practical-machine-learning + * + * see also "Sebastian Schelter, Christoph Boden, Volker Markl: + * Scalable Similarity-Based Neighborhood Methods with MapReduce + * ACM Conference on Recommender Systems 2012" + */ +object CooccurrenceAnalysis extends Serializable { + + /** Compares (Int,Double) pairs by the second value */ + private val orderByScore = Ordering.fromLessThan[(Int, Double)] { case ((_, score1), (_, score2)) => score1 > score2 } + + def cooccurrences(drmARaw: DrmLike[Int], randomSeed: Int = 0xdeadbeef, maxInterestingItemsPerThing: Int = 50, + maxNumInteractions: Int = 500, drmBs: Array[DrmLike[Int]] = Array()): List[DrmLike[Int]] = { + + implicit val disributedContext = drmARaw.context + + // Apply selective downsampling, pin resulting matrix + val drmA = sampleDownAndBinarize(drmARaw, randomSeed, maxNumInteractions).checkpoint() + + // num users, which equals the maximum number of interactions per item + val numUsers = drmA.nrow.toInt + + // Compute & broadcast the number of interactions per thing in A + val bcastInteractionsPerItemA = drmBroadcast(drmA.colSums) + + // Compute co-occurrence matrix A'A + val drmAtA = drmA.t %*% drmA + + // Compute loglikelihood scores and sparsify the resulting matrix to get the indicator matrix + val drmIndicatorsAtA = computeIndicators(drmAtA, numUsers, maxInterestingItemsPerThing, bcastInteractionsPerItemA, + bcastInteractionsPerItemA, crossCooccurrence = false) + + var indicatorMatrices = List(drmIndicatorsAtA) + + // Now look at cross-co-occurrences + for (drmBRaw <- drmBs) { + // Down-sample and pin other interaction matrix + val drmB = sampleDownAndBinarize(drmBRaw, randomSeed, maxNumInteractions).checkpoint() + + // Compute & broadcast the number of interactions per thing in B + val bcastInteractionsPerThingB = drmBroadcast(drmB.colSums) + + // Compute cross-co-occurrence matrix B'A + val drmBtA = drmB.t %*% drmA + + val drmIndicatorsBtA = computeIndicators(drmBtA, numUsers, maxInterestingItemsPerThing, + bcastInteractionsPerThingB, bcastInteractionsPerItemA) + + indicatorMatrices = indicatorMatrices :+ drmIndicatorsBtA + + drmB.uncache() + } + + // Unpin downsampled interaction matrix + drmA.uncache() + + // Return list of indicator matrices + indicatorMatrices + } + + /** + * Compute loglikelihood ratio + * see http://tdunning.blogspot.de/2008/03/surprise-and-coincidence.html for details + **/ + def loglikelihoodRatio(numInteractionsWithA: Long, numInteractionsWithB: Long, + numInteractionsWithAandB: Long, numInteractions: Long) = { + + val k11 = numInteractionsWithAandB + val k12 = numInteractionsWithA - numInteractionsWithAandB + val k21 = numInteractionsWithB - numInteractionsWithAandB + val k22 = numInteractions - numInteractionsWithA - numInteractionsWithB + numInteractionsWithAandB + + LogLikelihood.logLikelihoodRatio(k11, k12, k21, k22) + } + + def computeIndicators(drmBtA: DrmLike[Int], numUsers: Int, maxInterestingItemsPerThing: Int, + bcastNumInteractionsB: BCast[Vector], bcastNumInteractionsA: BCast[Vector], + crossCooccurrence: Boolean = true) = { + drmBtA.mapBlock() { + case (keys, block) => + + val llrBlock = block.like() + val numInteractionsB: Vector = bcastNumInteractionsB + val numInteractionsA: Vector = bcastNumInteractionsA + + for (index <- 0 until keys.size) { + + val thingB = keys(index) + + // PriorityQueue to select the top-k items + val topItemsPerThing = new mutable.PriorityQueue[(Int,Double)]()(orderByScore) + + block(index, ::).nonZeroes().foreach { elem => + val thingA = elem.index + val cooccurrences = elem.get + + // exclude co-occurrences of the item with itself + if (crossCooccurrence || thingB != thingA) { + // Compute loglikelihood ratio + val llrRatio = loglikelihoodRatio(numInteractionsB(thingB).toLong, numInteractionsA(thingA).toLong, + cooccurrences.toLong, numUsers) + val candidate = thingA -> llrRatio + + // Enqueue item with score, if belonging to the top-k + if (topItemsPerThing.size < maxInterestingItemsPerThing) { + topItemsPerThing.enqueue(candidate) + } else if (orderByScore.lt(candidate, topItemsPerThing.head)) { + topItemsPerThing.dequeue() + topItemsPerThing.enqueue(candidate) + } + } + } + + // Add top-k interesting items to the output matrix + topItemsPerThing.dequeueAll.foreach { case (otherThing, llrScore) => llrBlock(index, otherThing) = llrScore } + } + + keys -> llrBlock + } + } + + /** + * Selectively downsample users and things with an anormalous amount of interactions, inspired by + * https://github.com/tdunning/in-memory-cooccurrence/blob/master/src/main/java/com/tdunning/cooc/Analyze.java + * + * additionally binarizes input matrix, as we're only interesting in knowing whether interactions happened or not + */ + def sampleDownAndBinarize(drmM: DrmLike[Int], seed: Int, maxNumInteractions: Int) = { + + implicit val distributedContext = drmM.context + + // Pin raw interaction matrix + val drmI = drmM.checkpoint() + + // Broadcast vector containing the number of interactions with each thing + val bcastNumInteractions = drmBroadcast(drmI.colSums) + + val downSampledDrmI = drmI.mapBlock() { + case (keys, block) => + val numInteractions: Vector = bcastNumInteractions + + // Use a hash of the unique first key to seed the RNG, makes this computation repeatable in case of failures + val random = RandomUtils.getRandom(MurmurHash.hash(keys(0), seed)) + + val downsampledBlock = block.like() + + // Downsample the interaction vector of each user + for (userIndex <- 0 until keys.size) { + + val interactionsOfUser = block(userIndex, ::) + val numInteractionsOfUser = interactionsOfUser.sum + + val perUserSampleRate = math.min(maxNumInteractions, numInteractionsOfUser) / numInteractionsOfUser + + interactionsOfUser.nonZeroes().foreach { elem => + val numInteractionsWithThing = numInteractions(elem.index) + val perThingSampleRate = math.min(maxNumInteractions, numInteractionsWithThing) / numInteractionsWithThing + + if (random.nextDouble() <= math.min(perUserSampleRate, perThingSampleRate)) { + // We ignore the original interaction value and create a binary 0-1 matrix + // as we only consider whether interactions happened or did not happen + downsampledBlock(userIndex, elem.index) = 1 + } + } + } + + keys -> downsampledBlock + } + + // Unpin raw interaction matrix + drmI.uncache() + + downSampledDrmI + } +} diff --git a/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala b/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala new file mode 100644 index 0000000000..c640e1ef16 --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala @@ -0,0 +1,169 @@ +/* + * 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.mahout.cf.examples + +import scala.io.Source +import org.apache.mahout.math._ +import scalabindings._ +import RLikeOps._ +import drm._ +import RLikeDrmOps._ +import org.apache.mahout.sparkbindings._ + +import org.apache.mahout.cf.CooccurrenceAnalysis._ +import scala.collection.JavaConversions._ + +/** + * The Epinions dataset contains ratings from users to items and a trust-network between the users. + * We use co-occurrence analysis to compute "users who like these items, also like that items" and + * "users who trust these users, like that items" + * + * Download and unpack the dataset files from: + * + * http://www.trustlet.org/datasets/downloaded_epinions/ratings_data.txt.bz2 + * http://www.trustlet.org/datasets/downloaded_epinions/trust_data.txt.bz2 + **/ +object RunCrossCooccurrenceAnalysisOnEpinions { + + def main(args: Array[String]): Unit = { + + if (args.length == 0) { + println("Usage: RunCooccurrenceAnalysisOnMovielens1M ") + println("Download the dataset from http://www.trustlet.org/datasets/downloaded_epinions/ratings_data.txt.bz2 and") + println("http://www.trustlet.org/datasets/downloaded_epinions/trust_data.txt.bz2") + sys.exit(-1) + } + + val datasetDir = args(0) + + val epinionsRatings = new SparseMatrix(49290, 139738) + + var firstLineSkipped = false + for (line <- Source.fromFile(datasetDir + "/ratings_data.txt").getLines()) { + if (line.contains(' ') && firstLineSkipped) { + val tokens = line.split(' ') + val userID = tokens(0).toInt - 1 + val itemID = tokens(1).toInt - 1 + val rating = tokens(2).toDouble + epinionsRatings(userID, itemID) = rating + } + firstLineSkipped = true + } + + val epinionsTrustNetwork = new SparseMatrix(49290, 49290) + firstLineSkipped = false + for (line <- Source.fromFile(datasetDir + "/trust_data.txt").getLines()) { + if (line.contains(' ') && firstLineSkipped) { + val tokens = line.trim.split(' ') + val userID = tokens(0).toInt - 1 + val trustedUserId = tokens(1).toInt - 1 + epinionsTrustNetwork(userID, trustedUserId) = 1 + } + firstLineSkipped = true + } + + System.setProperty("spark.kryo.referenceTracking", "false") + System.setProperty("spark.kryoserializer.buffer.mb", "100") + +// implicit val distributedContext = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", +// customJars = Traversable.empty[String]) + implicit val distributedContext = mahoutSparkContext(masterUrl = "spark://occam4:7077", appName = "MahoutClusteredContext", + customJars = Traversable.empty[String]) + + val drmEpinionsRatings = drmParallelize(epinionsRatings, numPartitions = 2) + val drmEpinionsTrustNetwork = drmParallelize(epinionsTrustNetwork, numPartitions = 2) + + val indicatorMatrices = cooccurrences(drmEpinionsRatings, randomSeed = 0xdeadbeef, + maxInterestingItemsPerThing = 100, maxNumInteractions = 500, Array(drmEpinionsTrustNetwork)) +//hdfs://occam4:54310/user/pat/xrsj/ +/* + RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(0), + "/tmp/co-occurrence-on-epinions/indicators-item-item/") + RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(1), + "/tmp/co-occurrence-on-epinions/indicators-trust-item/") +*/ + RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(0), + "hdfs://occam4:54310/user/pat/xrsj/indicators-item-item/") + RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(1), + "hdfs://occam4:54310/user/pat/xrsj/indicators-trust-item/") + + distributedContext.close() + + println("Saved indicators to /tmp/co-occurrence-on-epinions/") + } +} + +/** + * The movielens1M dataset contains movie ratings, we use co-occurrence analysis to compute + * "users who like these movies, also like that movies" + * + * Download and unpack the dataset files from: + * http://files.grouplens.org/datasets/movielens/ml-1m.zip + */ +object RunCooccurrenceAnalysisOnMovielens1M { + + def main(args: Array[String]): Unit = { + + if (args.length == 0) { + println("Usage: RunCooccurrenceAnalysisOnMovielens1M ") + println("Download the dataset from http://files.grouplens.org/datasets/movielens/ml-1m.zip") + sys.exit(-1) + } + + val datasetDir = args(0) + + System.setProperty("spark.kryo.referenceTracking", "false") + System.setProperty("spark.kryoserializer.buffer.mb", "100") + + implicit val sc = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", + customJars = Traversable.empty[String]) + + System.setProperty("mahout.math.AtA.maxInMemNCol", 4000.toString) + + val movielens = new SparseMatrix(6040, 3952) + + for (line <- Source.fromFile(datasetDir + "/ratings.dat").getLines()) { + val tokens = line.split("::") + val userID = tokens(0).toInt - 1 + val itemID = tokens(1).toInt - 1 + val rating = tokens(2).toDouble + movielens(userID, itemID) = rating + } + + val drmMovielens = drmParallelize(movielens, numPartitions = 2) + + val indicatorMatrix = cooccurrences(drmMovielens).head + + RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrix, + "/tmp/co-occurrence-on-movielens/indicators-item-item/") + + sc.stop() + + println("Saved indicators to /tmp/co-occurrence-on-movielens/") + } +} + +object RecommendationExamplesHelper { + + def saveIndicatorMatrix(indicatorMatrix: DrmLike[Int], path: String) = { + indicatorMatrix.rdd.flatMap({ case (thingID, itemVector) => + for (elem <- itemVector.nonZeroes()) yield { thingID + '\t' + elem.index } + }) + .saveAsTextFile(path) + } +} From 74b9921c4c9bd8903585bbd74d9e66298ea8b7a0 Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 4 Jun 2014 13:09:07 -0700 Subject: [PATCH 02/38] Adding stuff for itemsimilarity driver for Spark --- spark/src/.DS_Store | Bin 6148 -> 0 bytes .../apache/mahout/drivers/FileSysUtils.scala | 34 +++ .../mahout/drivers/IndexedDataset.scala | 54 +++++ .../drivers/IndexedDatasetDriverTest.scala | 164 ++++++++++++++ .../mahout/drivers/ItemSimilarityDriver.scala | 208 ++++++++++++++++++ .../apache/mahout/drivers/MahoutDriver.scala | 74 +++++++ .../mahout/drivers/MahoutOptionParser.scala | 24 ++ .../apache/mahout/drivers/ReaderWriter.scala | 188 ++++++++++++++++ .../org/apache/mahout/drivers/Schema.scala | 30 +++ .../mahout/drivers/IndexedDatasetTest.scala | 25 +++ .../drivers/ItemSimilarityDriver$Test.scala | 180 +++++++++++++++ 11 files changed, 981 insertions(+) delete mode 100644 spark/src/.DS_Store create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/IndexedDatasetDriverTest.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/MahoutOptionParser.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/Schema.scala create mode 100644 spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala create mode 100644 spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala diff --git a/spark/src/.DS_Store b/spark/src/.DS_Store deleted file mode 100644 index 7b0d36729e2ee777a660f9e8c6709dd97bc2fb68..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6148 zcmeH~F>V4u3`M`g7D#EfOgRk)$PGpaPQV2S&>%r5BKkQx-ySzvsH0W%E!l7ES!;KH zv9ktX>-*^w7y&HlPOLmk%$N_j;tOYdpMH*)!|itQBJHgMp3+B5_H$d10#ZNMM4T9irw zDe%t}uwk>?toc%Twm!X{*Y{cVb)%DUIm54?049DEf6&9YUwlE + options.copy(input = x) + } text ("Path for input. It may be a filename or directory name and can be a local file path or an HDFS URI (required).") + opt[String]('o', "output") required() action { (x, options) => + options.copy(output = x) + } text ("Output will be in sub-directories stored here so this must be a directory path (required).") + note("\nAlgorithm control options:") + opt[String]("master") abbr ("ma") text ("URL for the Spark Master. (optional). Default: 'local'") action { (x, options) => + options.copy(master = x) + } + opt[Int]("maxPrefs") abbr ("mppu") action { (x, options) => + options.copy(maxPrefs = x) + } text ("Max number of preferences to consider per user or item, users or items with more preferences will be sampled down (optional). Default: 500") validate { x => + if (x > 0) success else failure("Option --maxPrefs must be > 0") + } + opt[Int]("minPrefs") abbr ("mp") action { (x, options) => + options.copy(minPrefs = x) + } text ("Ignore users with less preferences than this (optional). Default: 1") validate { x => + if (x > 0) success else failure("Option --minPrefs must be > 0") + } + + opt[Int]('m', "maxSimilaritiesPerItem") action { (x, options) => + options.copy(maxSimilaritiesPerItem = x) + } text ("Try to cap the number of similar items for each item to this number (optional). Default: 100") validate { x => + if (x > 0) success else failure("Option --maxSimilaritiesPerItem must be > 0") + } + opt[Int]("randomSeed") abbr ("rs") action { (x, options) => + options.copy(randomSeed = x) + } text ("Int to seed random number generator (optional). Default: Uses time to generate a seed") validate { x => + if (x > 0) success else failure("Option --randomSeed must be > 0") + } + note("\nInput text file schema options:") + opt[String]("inDelim") abbr ("d") text ("Input delimiter character (optional). Default: '\\t'") action { (x, options) => + options.copy(inDelim = x) + } + opt[String]("filter1") abbr ("f1") action { (x, options) => + options.copy(filter1 = x) + } text ("String whose presence indicates a datum for the primary item set, can be a regex (optional). Default: no filtered is applied, all is used") + opt[String]("filter2") abbr ("f2") action { (x, options) => + options.copy(filter2 = x) + } text ("String whose presence indicates a datum for the secondary item set, can be a regex (optional). Used in cross-cooccurrence. Default: no secondary filter is applied") + opt[Int]("rowIDPosition") abbr ("rc") action { (x, options) => + options.copy(rowIDPosition = x) + } text ("Column number (0 based Int) containing the row ID string (optional). Default: 0") validate { x => + if (x >= 0) success else failure("Option --rowIDColNum must be >= 0") + } + opt[Int]("itemIDPosition") abbr ("ic") action { (x, options) => + options.copy(itemIDPosition = x) + } text ("Column number (0 based Int) containing the item ID string (optional). Default: 1") validate { x => + if (x >= 0) success else failure("Option --itemIDColNum must be >= 0") + } + opt[Int]("filterPosition") abbr ("fc") action { (x, options) => + options.copy(filterPosition = x) + } text ("Column number (0 based Int) containing the filter string (optional). Default: -1 for no filter") validate { x => + if (x >= -1) success else failure("Option --filterColNum must be >= -1") + } + note("\nDefault input schema will accept: 'userIDitemId' or 'userIDitemIDany-text...' and all rows will be used") + note("\nFile input options:") + opt[Unit]('r', "recursive") action { (_, options) => + options.copy(recursive = true) + } text ("The input path should be searched recursively for files that match the filename pattern from -fp (optional), Default: false") + opt[String]("filenamePattern") abbr ("fp") action { (x, options) => + options.copy(filenamePattern = x) + } text ("Regex to match in determining input files (optional). Default: filename in the --input option or '^part-.*' if --input is a directory") + note("\nOutput text file schema options:") + opt[String]("outDelim1") abbr ("od1") action { (x, options) => + options.copy(outDelim1 = x) + } text ("Primary output inDelim value, used to separate row IDs from the similar items list (optional). Default: '\\t'") + opt[String]("outDelim2") abbr ("od2") action { (x, options) => + options.copy(outDelim2 = x) + } text ("Secondary output inDelim value, used to separate item IDs from their values in the similar items list (optional). Default: ':'") + opt[String]("outDelim3") abbr ("od3") action { (x, options) => + options.copy(outDelim3 = x) + } text ("Last inDelim value, used to separate (itemID:value) tuples in the similar items list. (optional). Default: ','") + note("\nDefault delimiters will produce output of the form: 'itemID1>itemID2:value2,itemID10:value10...'") + note("\nNote: Only the Log Likelihood Ratio (LLR) is supported as a similarity measure.\n") + help("help") abbr ("h") text ("prints this usage text\n") + checkConfig { c => + if (c.filterPosition == c.itemIDPosition || c.filterPosition == c.rowIDPosition || c.rowIDPosition == c.itemIDPosition) failure("The row, item, and filter positions must be unique.") else success + } + checkConfig { c => + if (c.filter1 != null && c.filter2 != null && c.filter1 == c.filter2) failure("If using filters they must be unique.") else success + } + } + parser.parse(args, Options()) map { opts => + options = opts + process + } + } + + private def readIndexedDatasets: Array[IndexedDataset] = { + val inFiles = FileSysUtils(options.input, options.filenamePattern, options.recursive ).uris + if(inFiles.isEmpty){Array()}else{ + val indexedDataset1 = IndexedDataset(readStore1.readFrom(inFiles)) + if (options.filterPosition != -1 && options.filter2 != null) { + val indexedDataset2 = IndexedDataset(readStore2.readFrom(inFiles)) + Array(indexedDataset1, indexedDataset2) + } else { + Array(indexedDataset1) + } + } + } + + override def start(masterUrl: String = options.master, appName: String = options.appName, + customJars:Traversable[String] = Traversable.empty[String]): Unit = { + //todo: create and modify a SparkContext here, which can be passed in to mahoutSparkContext in the super.start + System.setProperty("spark.kryo.referenceTracking", "false") + System.setProperty("spark.kryoserializer.buffer.mb", "100") + System.setProperty("spark.executor.memory", "2g") + super.start(masterUrl, appName, customJars) + val readSchema1 = new Schema("delim" -> options.inDelim, "filter" -> options.filter1, "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, "filterPosition" -> options.filterPosition) + if(options.filterPosition != -1 && options.filter2 != null) { + val readSchema2 = new Schema("delim" -> options.inDelim, "filter" -> options.filter2, "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, "filterPosition" -> options.filterPosition) + readStore2 = new TextDelimitedIndexedDatasetReader(readSchema2, mc) + } + val writeSchema = new Schema("delim1" -> options.outDelim1, "delim2" -> options.outDelim2, "delim3" -> options.outDelim3) + readStore1 = new TextDelimitedIndexedDatasetReader(readSchema1, mc) + writeStore = new TextDelimitedIndexedDatasetWriter(writeSchema, mc) + } + + override def process: Unit = { + start() + + val indexedDatasets = readIndexedDatasets + + val indicatorMatrices = CooccurrenceAnalysis.cooccurrences(indexedDatasets(0).matrix, options.randomSeed, options.maxSimilaritiesPerItem, options.maxPrefs, Array(indexedDatasets(1).matrix)) + + val selfIndicatorDataset = new IndexedDataset(indicatorMatrices(0), indexedDatasets(0).columnIDs, indexedDatasets(0).columnIDs) // self similarity + + writeStore.writeTo(selfIndicatorDataset, options.output+"indicator-matrix") + if(indexedDatasets.length > 1){ + val crossIndicatorDataset = new IndexedDataset(indicatorMatrices(1), indexedDatasets(0).columnIDs, indexedDatasets(1).columnIDs) // cross similarity + writeStore.writeTo(crossIndicatorDataset, options.output+"cross-indicator-matrix") + } + + stop + } + + //Default values go here, any '_' or null should be 'required' in the Parser or flags an unused option + // todo: support two input streams for cross-similarity, maybe assume one schema for both inputs + case class Options( + master: String = "local", + appName: String = "ItemSimilarityJob", + randomSeed: Int = System.currentTimeMillis().toInt, + recursive: Boolean = false, + input: String = null, + output: String = null, + filenamePattern: String = "^part-.*", + maxSimilaritiesPerItem: Int = 100, + maxPrefs: Int = 500, + minPrefs: Int = 1, + rowIDPosition: Int = 0, + itemIDPosition: Int = 1, + filterPosition: Int = -1, + filter1: String = null, + filter2: String = null, + inDelim: String = ",", + outDelim1: String = "\t", + outDelim2: String = ":", + outDelim3: String = "," + ) + +} diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala new file mode 100644 index 0000000000..b8a79f8cac --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala @@ -0,0 +1,74 @@ +/* + * 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.mahout.drivers + +import org.apache.spark.SparkContext +import org.apache.mahout.sparkbindings._ + +/** Extend this class to create a Mahout CLI driver. Minimally you must override process and main. Also define a command line parser and default options or fill in the following template: + * {{{ + * object SomeDriver extends MahoutDriver { + * override def main(args: Array[String]): Unit = { + * val parser = new MahoutOptionParser[Options]("Job Name") { + * head("Job Name", "Spark") + * note("Various CLI options") + * //see https://github.com/scopt/scopt for a good Scala option parser, which MahoutOptionParser extends + * } + * parser.parse(args, Options()) map { opts => + * options = opts + * process + * } + * } + * + * override def process: Unit = { + * start() + * //don't just stand there do something + * stop + * } + * + * //Default values go here, any '_' or null should be 'required' in the Parser or flags an unused option + * case class Options( + * appName: String = "Job Name", ... + * ) + * } + * }}} + */ +abstract class MahoutDriver { + protected var mc: SparkContext = _ + /** Creates a Spark context to run the job inside. + * Creates a Spark context to run the job inside. Override to set the SparkConf values specific to the job, these must be set before the context is created. + * @param masterUrl Spark master URL + * @param appName Name to display in Spark UI + * @param customJars List of paths to custom jars + * */ + protected def start(masterUrl: String, appName: String, + customJars:Traversable[String] = Traversable.empty[String]) : Unit = { + mc = mahoutSparkContext(masterUrl, appName, customJars) + } + + /** Override (optionally) for special cleanup */ + protected def stop: Unit = { + mc.stop + } + + /** This is wher you do the work, call start first, then before exiting call stop */ + protected def process: Unit + + /** Parse command line and call process*/ + def main(args: Array[String]): Unit +} diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutOptionParser.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutOptionParser.scala new file mode 100644 index 0000000000..8a337f50a6 --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutOptionParser.scala @@ -0,0 +1,24 @@ +/* + * 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.mahout.drivers + +import scopt.OptionParser + +/** Modifies default [[scopt.OptionParser]] to output long help-like usage + error message */ +class MahoutOptionParser[C](programName: String) extends OptionParser[C](programName: String) { + override def showUsageOnError = true +} diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala new file mode 100644 index 0000000000..1179eef2ee --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -0,0 +1,188 @@ +/* + * 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.mahout.drivers + +import org.apache.mahout.sparkbindings.drm._ +import org.apache.spark.SparkContext._ +import org.apache.mahout.math.RandomAccessSparseVector +import org.apache.spark.SparkContext +import com.google.common.collect.{BiMap, HashBiMap} +import scala.collection.JavaConversions._ +import org.apache.mahout.sparkbindings.DrmRdd +import org.apache.mahout.math.drm.DrmLike + + +/** Reader trait is abstract in the sense that the reader function must be defined by an extending trait, which also defines the type to be read. + * @tparam T type of object read, usually supplied by an extending trait. + * @todo the reader need not create both dictionaries but does at present. There are cases where one or the other dictionary is never used so saving the memory for a very large dictionary may be worth the optimization to specify which dictionaries are created. + */ +trait Reader[T]{ + val sc: SparkContext + val readSchema: Schema + protected def reader(sc: SparkContext, readSchema: Schema, source: String): T + def readFrom(source: String): T = reader(sc, readSchema, source) +} + +/** Writer trait is abstract in the sense that the writer method must be supplied by an extending trait, which also defines the type to be written. + * @tparam T + */ +trait Writer[T]{ + val sc: SparkContext + val writeSchema: Schema + protected def writer(sc: SparkContext, writeSchema: Schema, dest: String, collection: T): Unit + def writeTo(collection: T, dest: String) = writer(sc, writeSchema, dest, collection) +} + +/** Extends Reader trait to supply the [[org.apache.mahout.drivers.IndexedDataset]] as the type read and a reader function for reading text delimited files as described in the [[org.apache.mahout.drivers.Schema]] + */ +trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ + /** Read in text delimited tuples from all URIs in this comma delimited source String. + * + * @param sc context for the Spark job + * @param readSchema describes the delimiters and positions of values in the text delimited file. + * @param source comma delimited URIs of text files to be read into the [[org.apache.mahout.drivers.IndexedDataset]] + * @return + */ + protected def reader(sc: SparkContext, readSchema: Schema, source: String): IndexedDataset = { + try { + val delimiter = readSchema("delim").asInstanceOf[String] + val rowIDPosition = readSchema("rowIDPosition").asInstanceOf[Int] + val columnIDPosition = readSchema("columnIDPosition").asInstanceOf[Int] + val filterPosition = readSchema("filterPosition").asInstanceOf[Int] + val filterBy = readSchema("filter").asInstanceOf[String] + //instance vars must be put into locally scoped vals when used in closures that are + //executed but Spark + + assert(!source.isEmpty, { + println(this.getClass.toString + ": has no files to read") + throw new IllegalArgumentException + }) + + var columns = sc.textFile(source).map({ line => line.split(delimiter)}) + + columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) + + val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) + + interactions.cache() + + val rowIDs = interactions.map({ case (rowID, _) => rowID}).distinct().collect() + val columnIDs = interactions.map({ case (_, columnID) => columnID}).distinct().collect() + + val numRows = rowIDs.size + val numColumns = columnIDs.size + + val rowIDDictionary = asOrderedDictionary(rowIDs) + val columnIDDictionary = asOrderedDictionary(columnIDs) + + val rowIDDictionary_bcast = sc.broadcast(rowIDDictionary) + val columnIDDictionary_bcast = sc.broadcast(columnIDDictionary) + + val indexedInteractions = + interactions.map({ case (rowID, columnID) => + val rowIndex = rowIDDictionary_bcast.value.get(rowID).get + val columnIndex = columnIDDictionary_bcast.value.get(columnID).get + + rowIndex -> columnIndex + }).groupByKey().map({ case (rowIndex, columnIndexes) => + val row = new RandomAccessSparseVector(numColumns) + for (columnIndex <- columnIndexes) { + row.setQuick(columnIndex, 1.0) + } + rowIndex -> row + }).asInstanceOf[DrmRdd[Int]] + + //todo: old API, val drmInteractions = new CheckpointedDrmBase[Int](indexedInteractions, numRows, numColumns) + val drmInteractions = new CheckpointedDrmSpark[Int](indexedInteractions, numRows, numColumns) + + IndexedDataset(drmInteractions, rowIDDictionary, columnIDDictionary) + + } catch { + case cce: ClassCastException => { + println(this.getClass.toString + ": Schema has illegal values"); throw cce + } + } + } + + private def asOrderedDictionary(entries: Array[String]): BiMap[String, Int] = { + var dictionary: BiMap[String, Int] = HashBiMap.create() + var index = 0 + for (entry <- entries) { + dictionary.forcePut(entry, index) + index += 1 + } + dictionary + } +} + +trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ + /** Read in text delimited tuples from all URIs in this comma delimited source String. + * + * @param sc context for the Spark job + * @param writeSchema describes the delimiters and positions of values in the output text delimited file. + * @param dest directory to write text delimited version of [[org.apache.mahout.drivers.IndexedDataset]] + */ + protected def writer(sc: SparkContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { + try { + val outDelim1 = writeSchema("delim1").asInstanceOf[String] + val outDelim2 = writeSchema("delim2").asInstanceOf[String] + val outDelim3 = writeSchema("delim3").asInstanceOf[String] + //instance vars must be put into locally scoped vals when put into closures that are + //executed but Spark + assert (indexedDataset != null, {println(this.getClass.toString+": has no indexedDataset to write"); throw new IllegalArgumentException }) + assert (!dest.isEmpty, {println(this.getClass.toString+": has no destination or indextedDataset to write"); throw new IllegalArgumentException}) + val matrix: DrmLike[Int] = indexedDataset.matrix + val rowIDDictionary: BiMap[String, Int] = indexedDataset.rowIDs + val columnIDDictionary: BiMap[String, Int] = indexedDataset.columnIDs + matrix.rdd.map({ case (rowID, itemVector) => + var line: String = rowIDDictionary.inverse.get(rowID) + outDelim1 + for (item <- itemVector.nonZeroes()) { + line += columnIDDictionary.inverse.get(item.index) + outDelim2 + item.get + outDelim3 + } + line.dropRight(1) + }) + .saveAsTextFile(dest) + }catch{ + case cce: ClassCastException => {println(this.getClass.toString+": Schema has illegal values"); throw cce} + } + } +} + +/** A combined trait that reads and writes */ +trait TDIndexedDatasetReaderWriter extends TDIndexedDatasetReader with TDIndexedDatasetWriter + +/** Reads text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. + * @param readSchema describes the delimiters and position of values in the text delimited file to be read. + * @param sc Spark context for reading files + * @note The source files are supplied to the readFrom trait method. + * */ +class TextDelimitedIndexedDatasetReader(val readSchema: Schema, val sc: SparkContext) extends TDIndexedDatasetReader + +/** Writes text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. + * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. + * @param sc Spark context for reading files + * @note the destination if supplied to the writeTo trait method + * */ +class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema, val sc: SparkContext) extends TDIndexedDatasetWriter + +/** Reads and writes text delimited files to/from an IndexedDataset. Classes are needed to supply trait params in their constructor. + * @param readSchema describes the delimiters and position of values in the text delimited file(s) to be read. + * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. + * @param sc Spark context for reading the files + * */ +class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema, val sc: SparkContext) extends TDIndexedDatasetReaderWriter diff --git a/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala b/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala new file mode 100644 index 0000000000..13fa292b92 --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala @@ -0,0 +1,30 @@ + +/* + * 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.mahout.drivers + +import scala.collection.mutable.HashMap + +/** Syntactic sugar for HashMap[String, Any] + * + * @param params list of mappings for instantiation {{{val mySchema = new Schema("one" -> 1, "two" -> "2", ...)}}} + */ +class Schema(params: Tuple2[String, Any]*) extends HashMap[String, Any] { + //todo: this require a mutable HashMap, do we care? + this ++= params +} diff --git a/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala b/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala new file mode 100644 index 0000000000..fb13361090 --- /dev/null +++ b/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala @@ -0,0 +1,25 @@ +package org.apache.mahout.drivers + +import org.scalatest.FunSuite + +/* + * 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. + */ + +class IndexedDatasetTest extends FunSuite { + //todo: put some tests here! + +} diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala new file mode 100644 index 0000000000..c1ae4ab5a3 --- /dev/null +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala @@ -0,0 +1,180 @@ +package org.apache.mahout.drivers + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.mahout.sparkbindings._ +import java.io.{FileWriter, BufferedWriter} +import com.google.common.io.Closeables +import org.apache.spark.SparkContext + +/* + * 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. + */ + +class ItemSimilarityDriver$Test extends FunSuite { + private var sc: SparkContext = _ + + + test ("Running some text delimited files through the import/cooccurrence/export"){ + + val exampleCsvlogStatements = Array( + "12569537329,user1,item1,\"view\"", + "12569537329,user1,item2,\"view\"", + "12569537329,user1,item2,\"like\"", + "12569537329,user1,item3,\"like\"", + "12569537329,user2,item2,\"view\"", + "12569537329,user2,item2,\"like\"", + "12569537329,user3,item1,\"like\"", + "12569537329,user3,item3,\"view\"", + "12569537329,user3,item3,\"like\"" + ) + + val exampleTsvLogStatements = Array( + "12569537329\tuser1\titem1\t\"view\"", + "12569537329\tuser1\titem2\t\"view\"", + "12569537329\tuser1\titem2\t\"like\"", + "12569537329\tuser1\titem3\t\"like\"", + "12569537329\tuser2\titem2\t\"view\"", + "12569537329\tuser2\titem2\t\"like\"", + "12569537329\tuser3\titem1\t\"like\"", + "12569537329\tuser3\titem3\t\"view\"", + "12569537329\tuser3\titem3\t\"like\"" + ) + + + val csvLogStatements = Array( + "u1,purchase,iphone", + "u1,purchase,ipad", + "u2,purchase,nexus", + "u2,purchase,galaxy", + "u3,purchase,surface", + "u4,purchase,iphone", + "u4,purchase,galaxy", + "u1,view,iphone", + "u1,view,ipad", + "u1,view,nexus", + "u1,view,galaxy", + "u2,view,iphone", + "u2,view,ipad", + "u2,view,nexus", + "u2,view,galaxy", + "u3,view,surface", + "u3,view,nexus", + "u4,view,iphone", + "u4,view,ipad", + "u4,view,galaxy" + ) + + val tsvLogStatements = Array( + "u1\tpurchase\tiphone", + "u1\tpurchase\tipad", + "u2\tpurchase\tnexus", + "u2\tpurchase\tgalaxy", + "u3\tpurchase\tsurface", + "u4\tpurchase\tiphone", + "u4\tpurchase\tgalaxy", + "u1\tview\tiphone", + "u1\tview\tipad", + "u1\tview\tnexus", + "u1\tview\tgalaxy", + "u2\tview\tiphone", + "u2\tview\tipad", + "u2\tview\tnexus", + "u2\tview\tgalaxy", + "u3\tview\tsurface", + "u3\tview\tnexus", + "u4\tview\tiphone", + "u4\tview\tipad", + "u4\tview\tgalaxy" + ) + + var w: BufferedWriter = null + //try { + w = new BufferedWriter(new FileWriter("/tmp/cf-data.txt")) + w.write(csvLogStatements.mkString("\n")) + //} finally { + Closeables.close(w, false) + //} + /* + val indexedLikes = IndexedDatasetStore.readTuples(sc, "tmp/cf-data.txt", 2, ",", 0, 2, 1, "purchase") + + val indexedViews = IndexedDatasetStore.readTuples(sc, "/tmp/cf-data.txt", 2, ",", 0, 2, 1, "view") + + val drmLikes = indexedLikes.matrix + val drmViews = indexedViews.matrix + + // Now we could run cooccurrence analysis using the DRMs, instead we'll just fetch and print the matrices + val drmXCooccurrences = cooccurrences(drmLikes, randomSeed = 0xdeadbeef, + maxInterestingItemsPerThing = 100, maxNumInteractions = 500, Array(drmViews)) + + val inCoreViews = drmViews.collect + val inCoreLikes = drmLikes.collect + val inCoreIndicator = drmXCooccurrences(0).collect + val inCoreXIndicator = drmXCooccurrences(1).collect + println("\nLIKES:") + println(inCoreLikes) + println("\nVIEWS:") + println(inCoreViews) + println("\nINDICATOR MATRIX") + println(inCoreIndicator) + println("\nCROSS INDICATOR MATRIX") + println(inCoreXIndicator) + */ + + /* + //Clustered Spark and HDFS + ItemSimilarityDriver.main(Array( + "--input", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/cf-data.txt", + "--output", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/indicatorMatrices/", + "--master", "spark://occam4:7077", + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1" + )) + */ + //local multi-threaded Spark with HDFS using large dataset + /* ItemSimilarityDriver.main(Array( + "--input", "hdfs://occam4:54310/user/pat/xrsj/ratings_data.txt", + "--output", "hdfs://occam4:54310/user/pat/xrsj/indicatorMatrices/", + "--master", "local[4]", + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1" + )) + */ + + //local multi-threaded Spark with local FS + ItemSimilarityDriver.main(Array( + "--input", "/tmp/cf-data.txt", + "--output", "tmp/indicatorMatrices/", + "--master", "local[4]", + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1" + )) + + } + +} From a59265931ed3a51ba81e1a0a7171ebb102be4fa4 Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 4 Jun 2014 13:13:13 -0700 Subject: [PATCH 03/38] added scopt to pom deps --- spark/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/spark/pom.xml b/spark/pom.xml index ac99ffd936..1f08d9a1ed 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -288,6 +288,13 @@ + + + com.github.scopt + scopt_2.10 + 3.2.0 + + org.apache.spark From 16c03f7fa73c156859d1dba3a333ef9e8bf922b0 Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 4 Jun 2014 14:32:18 -0700 Subject: [PATCH 04/38] added Sebastian's MurmurHash changes Signed-off-by: pferrel --- .../java/org/apache/mahout/math/MurmurHash.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/math/src/main/java/org/apache/mahout/math/MurmurHash.java b/math/src/main/java/org/apache/mahout/math/MurmurHash.java index 0b3fab0675..32dfdd614e 100644 --- a/math/src/main/java/org/apache/mahout/math/MurmurHash.java +++ b/math/src/main/java/org/apache/mahout/math/MurmurHash.java @@ -17,6 +17,8 @@ package org.apache.mahout.math; +import com.google.common.primitives.Ints; + import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -29,7 +31,16 @@ */ public final class MurmurHash { - private MurmurHash() { + private MurmurHash() {} + + /** + * Hashes an int. + * @param data The int to hash. + * @param seed The seed for the hash. + * @return The 32 bit hash of the bytes in question. + */ + public static int hash(int data, int seed) { + return hash(ByteBuffer.wrap(Ints.toByteArray(data)), seed); } /** From 2f87f5433f90fa2c49ef386ca245943e1fc73beb Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 4 Jun 2014 18:44:16 -0700 Subject: [PATCH 05/38] MAHOUT-1541 still working on this, some refactoring in the DSL for abstracting away Spark has moved access to rddsno Jira is closed yet --- .../main/scala/org/apache/mahout/drivers/ReaderWriter.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala index 1179eef2ee..9201c81d65 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -149,6 +149,10 @@ trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ val matrix: DrmLike[Int] = indexedDataset.matrix val rowIDDictionary: BiMap[String, Int] = indexedDataset.rowIDs val columnIDDictionary: BiMap[String, Int] = indexedDataset.columnIDs + // below doesn't compile because the rdd is not in a CheckpointedDrmSpark also I don't know how to turn a + // CheckpointedDrmSpark[Int] into a DrmLike[Int], which I need to pass in the CooccurrenceAnalysis#cooccurrence + // This seems to be about the refacotring to abstract away from Spark but the Read and Write are Spark specific + // and the non-specific DrmLike is no longer attached to a CheckpointedDrmSpark, could be missing something though matrix.rdd.map({ case (rowID, itemVector) => var line: String = rowIDDictionary.inverse.get(rowID) + outDelim1 for (item <- itemVector.nonZeroes()) { From c6adaa44c80bba99d41600e260bbb1ad5c972e69 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 5 Jun 2014 09:52:23 -0700 Subject: [PATCH 06/38] MAHOUT-1464 import cleanup, minor changes to examples for running on Spark Cluster --- .../apache/mahout/cf/CooccurrenceAnalysis.scala | 2 -- .../mahout/cf/examples/Recommendations.scala | 17 ++++++++++------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index 5df329b3be..1f399ec947 100644 --- a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -23,11 +23,9 @@ import RLikeOps._ import drm._ import RLikeDrmOps._ import org.apache.mahout.sparkbindings._ - import scala.collection.JavaConversions._ import org.apache.mahout.math.stats.LogLikelihood import collection._ -// import scala.collection.parallel.mutable import org.apache.mahout.common.RandomUtils diff --git a/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala b/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala index c640e1ef16..afd6701605 100644 --- a/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala +++ b/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala @@ -79,29 +79,32 @@ object RunCrossCooccurrenceAnalysisOnEpinions { System.setProperty("spark.kryo.referenceTracking", "false") System.setProperty("spark.kryoserializer.buffer.mb", "100") +/* to run on local, can provide number of core by changing to local[4] */ + implicit val distributedContext = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", + customJars = Traversable.empty[String]) -// implicit val distributedContext = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", -// customJars = Traversable.empty[String]) + /* to run on a Spark cluster provide the Spark Master URL implicit val distributedContext = mahoutSparkContext(masterUrl = "spark://occam4:7077", appName = "MahoutClusteredContext", customJars = Traversable.empty[String]) - +*/ val drmEpinionsRatings = drmParallelize(epinionsRatings, numPartitions = 2) val drmEpinionsTrustNetwork = drmParallelize(epinionsTrustNetwork, numPartitions = 2) val indicatorMatrices = cooccurrences(drmEpinionsRatings, randomSeed = 0xdeadbeef, maxInterestingItemsPerThing = 100, maxNumInteractions = 500, Array(drmEpinionsTrustNetwork)) -//hdfs://occam4:54310/user/pat/xrsj/ -/* + +/* local storage */ RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(0), "/tmp/co-occurrence-on-epinions/indicators-item-item/") RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(1), "/tmp/co-occurrence-on-epinions/indicators-trust-item/") -*/ + +/* To run on HDFS put your path to the data here, example of fully qualified path on my cluster provided RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(0), "hdfs://occam4:54310/user/pat/xrsj/indicators-item-item/") RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(1), "hdfs://occam4:54310/user/pat/xrsj/indicators-trust-item/") - +*/ distributedContext.close() println("Saved indicators to /tmp/co-occurrence-on-epinions/") From a2f84dea3f32d3df3e98c61f085bc1fabd453551 Mon Sep 17 00:00:00 2001 From: pferrel Date: Sat, 7 Jun 2014 14:27:06 -0700 Subject: [PATCH 07/38] drmWrap seems to be the answer to the changed DrmLike interface. Code works again but more to do. --- .../main/scala/org/apache/mahout/drivers/IndexedDataset.scala | 4 ++-- .../main/scala/org/apache/mahout/drivers/ReaderWriter.scala | 4 +++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala index 36bda9022e..496db886a6 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala @@ -18,7 +18,7 @@ package org.apache.mahout.drivers import com.google.common.collect.BiMap -import org.apache.mahout.math.drm.DrmLike +import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} /** * Wraps a [[org.apache.mahout.sparkbindings.drm.DrmLike]] object with two [[com.google.common.collect.BiMap]]s to store ID/label translation dictionaries. @@ -37,7 +37,7 @@ import org.apache.mahout.math.drm.DrmLike * ID to and from the ordinal Mahout Int ID. This one holds column labels */ -case class IndexedDataset(matrix: DrmLike[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int]) { +case class IndexedDataset(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int]) { } /** diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala index 9201c81d65..af01ff0127 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -25,6 +25,7 @@ import com.google.common.collect.{BiMap, HashBiMap} import scala.collection.JavaConversions._ import org.apache.mahout.sparkbindings.DrmRdd import org.apache.mahout.math.drm.DrmLike +import org.apache.mahout.sparkbindings._ /** Reader trait is abstract in the sense that the reader function must be defined by an extending trait, which also defines the type to be read. @@ -108,7 +109,8 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ }).asInstanceOf[DrmRdd[Int]] //todo: old API, val drmInteractions = new CheckpointedDrmBase[Int](indexedInteractions, numRows, numColumns) - val drmInteractions = new CheckpointedDrmSpark[Int](indexedInteractions, numRows, numColumns) + //val drmInteractions = new CheckpointedDrmSpark[Int](indexedInteractions, numRows, numColumns) + val drmInteractions = drmWrap[Int](indexedInteractions, numRows, numColumns) IndexedDataset(drmInteractions, rowIDDictionary, columnIDDictionary) From 4b2fb07b21a8ac2d532ee51b65b27d1482293cb0 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 19 Jun 2014 10:08:02 -0700 Subject: [PATCH 08/38] for high level review, not ready for merge --- .../mahout/drivers/IndexedDataset.scala | 2 + .../drivers/IndexedDatasetDriverTest.scala | 164 ---------------- .../mahout/drivers/ItemSimilarityDriver.scala | 152 +++++++++------ .../apache/mahout/drivers/MahoutDriver.scala | 14 +- .../apache/mahout/drivers/ReaderWriter.scala | 90 ++++++--- .../org/apache/mahout/drivers/Schema.scala | 3 +- .../mahout/drivers/IndexedDatasetTest.scala | 10 +- .../drivers/ItemSimilarityDriver$Test.scala | 184 +++++++----------- 8 files changed, 243 insertions(+), 376 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/mahout/drivers/IndexedDatasetDriverTest.scala diff --git a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala index 496db886a6..bd473daec1 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala @@ -35,6 +35,8 @@ import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} * and from the ordinal Mahout Int ID. This one holds row labels * @param columnIDs BiMap[String, Int] storing a bidirectional mapping of external String * ID to and from the ordinal Mahout Int ID. This one holds column labels + * @todo Often no need for both or perhaps either dictionary, so save resources by allowing + * to be not created when not needed. */ case class IndexedDataset(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int]) { diff --git a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDatasetDriverTest.scala b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDatasetDriverTest.scala deleted file mode 100644 index 80fc134a6c..0000000000 --- a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDatasetDriverTest.scala +++ /dev/null @@ -1,164 +0,0 @@ -package org.apache.mahout.drivers - -import org.apache.mahout.sparkbindings._ -import java.io.{FileWriter, BufferedWriter} -import com.google.common.io.Closeables - -object ItemSimilarityDriverTest { - - def main(args: Array[String]): Unit = { - - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "100") - - implicit val sc = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", - customJars = Traversable.empty[String]) - - val exampleCsvlogStatements = Array( - "12569537329,user1,item1,\"view\"", - "12569537329,user1,item2,\"view\"", - "12569537329,user1,item2,\"like\"", - "12569537329,user1,item3,\"like\"", - "12569537329,user2,item2,\"view\"", - "12569537329,user2,item2,\"like\"", - "12569537329,user3,item1,\"like\"", - "12569537329,user3,item3,\"view\"", - "12569537329,user3,item3,\"like\"" - ) - - val exampleTsvLogStatements = Array( - "12569537329\tuser1\titem1\t\"view\"", - "12569537329\tuser1\titem2\t\"view\"", - "12569537329\tuser1\titem2\t\"like\"", - "12569537329\tuser1\titem3\t\"like\"", - "12569537329\tuser2\titem2\t\"view\"", - "12569537329\tuser2\titem2\t\"like\"", - "12569537329\tuser3\titem1\t\"like\"", - "12569537329\tuser3\titem3\t\"view\"", - "12569537329\tuser3\titem3\t\"like\"" - ) - - - val csvLogStatements = Array( - "u1,purchase,iphone", - "u1,purchase,ipad", - "u2,purchase,nexus", - "u2,purchase,galaxy", - "u3,purchase,surface", - "u4,purchase,iphone", - "u4,purchase,galaxy", - "u1,view,iphone", - "u1,view,ipad", - "u1,view,nexus", - "u1,view,galaxy", - "u2,view,iphone", - "u2,view,ipad", - "u2,view,nexus", - "u2,view,galaxy", - "u3,view,surface", - "u3,view,nexus", - "u4,view,iphone", - "u4,view,ipad", - "u4,view,galaxy" - ) - - val tsvLogStatements = Array( - "u1\tpurchase\tiphone", - "u1\tpurchase\tipad", - "u2\tpurchase\tnexus", - "u2\tpurchase\tgalaxy", - "u3\tpurchase\tsurface", - "u4\tpurchase\tiphone", - "u4\tpurchase\tgalaxy", - "u1\tview\tiphone", - "u1\tview\tipad", - "u1\tview\tnexus", - "u1\tview\tgalaxy", - "u2\tview\tiphone", - "u2\tview\tipad", - "u2\tview\tnexus", - "u2\tview\tgalaxy", - "u3\tview\tsurface", - "u3\tview\tnexus", - "u4\tview\tiphone", - "u4\tview\tipad", - "u4\tview\tgalaxy" - ) - - var w: BufferedWriter = null - try { - w = new BufferedWriter(new FileWriter("tmp/cf-data.txt")) - w.write(csvLogStatements.mkString("\n")) - } finally { - Closeables.close(w, false) - } - /* - val indexedLikes = IndexedDatasetStore.readTuples(sc, "tmp/cf-data.txt", 2, ",", 0, 2, 1, "purchase") - - val indexedViews = IndexedDatasetStore.readTuples(sc, "tmp/cf-data.txt", 2, ",", 0, 2, 1, "view") - - val drmLikes = indexedLikes.matrix - val drmViews = indexedViews.matrix - - // Now we could run cooccurrence analysis using the DRMs, instead we'll just fetch and print the matrices - val drmXCooccurrences = cooccurrences(drmLikes, randomSeed = 0xdeadbeef, - maxInterestingItemsPerThing = 100, maxNumInteractions = 500, Array(drmViews)) - - val inCoreViews = drmViews.collect - val inCoreLikes = drmLikes.collect - val inCoreIndicator = drmXCooccurrences(0).collect - val inCoreXIndicator = drmXCooccurrences(1).collect - println("\nLIKES:") - println(inCoreLikes) - println("\nVIEWS:") - println(inCoreViews) - println("\nINDICATOR MATRIX") - println(inCoreIndicator) - println("\nCROSS INDICATOR MATRIX") - println(inCoreXIndicator) - */ - sc.stop() -/* - //Clustered Spark and HDFS - ItemSimilarityDriver.main(Array( - "--input", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/cf-data.txt", - "--output", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/indicatorMatrices/", - "--master", "spark://occam4:7077", - "--filter1", "purchase", - "--filter2", "view", - "--inDelim", ",", - "--itemIDPosition", "2", - "--rowIDPosition", "0", - "--filterPosition", "1" - )) -*/ - //local multi-threaded Spark with HDFS -/* ItemSimilarityDriver.main(Array( - "--input", "hdfs://occam4:54310/user/pat/xrsj/ratings_data.txt", - "--output", "hdfs://occam4:54310/user/pat/xrsj/indicatorMatrices/", - "--master", "local[4]", - "--filter1", "purchase", - "--filter2", "view", - "--inDelim", ",", - "--itemIDPosition", "2", - "--rowIDPosition", "0", - "--filterPosition", "1" - )) -*/ - - //local multi-threaded Spark with local FS - ItemSimilarityDriver.main(Array( - "--input", "tmp/cf-data.txt", - "--output", "tmp/indicatorMatrices/", - "--master", "local[4]", - "--filter1", "purchase", - "--filter2", "view", - "--inDelim", ",", - "--itemIDPosition", "2", - "--rowIDPosition", "0", - "--filterPosition", "1" - )) - - - } -} diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index c72a3b3c8a..4b86f42dbf 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -20,25 +20,27 @@ package org.apache.mahout.drivers import org.apache.mahout.cf.CooccurrenceAnalysis /** - * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences()]]. - * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences()]]. It text lines - * that contain (row id, column id, ...). The IDs are user specified strings which will be preserved in the - * output. The individual tuples will be accumulated into a matrix and [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences()]] - * will be used to calculate row-wise self-similarity, or when using filters, will generate two - * matrices and calculate both the self similarity of the primary matrix and the row-wise similarity of the primary - * to the secondary. Returns one or two directories of text files formatted as specified in the options. - * @todo Should also take two input streams and do cross similarity with not filter required. - */ + * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. + * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. It reads text lines + * that contain (row id, column id, ...). The IDs are user specified strings which will be preserved in the + * output. The individual tuples will be accumulated into a matrix and [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]] + * will be used to calculate row-wise self-similarity, or when using filters, will generate two + * matrices and calculate both the self similarity of the primary matrix and the row-wise similarity of the primary + * to the secondary. Returns one or two directories of text files formatted as specified in the options. + */ object ItemSimilarityDriver extends MahoutDriver { + //todo: Should also take two input streams and do cross similarity with no filter required. + //todo: Required: should work on the old Mahout TDF that allows values (but ignores them), required for examples private var options: Options = _ private var readStore1: TextDelimitedIndexedDatasetReader = _ private var readStore2: TextDelimitedIndexedDatasetReader = _ private var writeStore: TextDelimitedIndexedDatasetWriter = _ + private var writeSchema: Schema = _ -/** - * @param args Command line args, if empty a help message is printed. - */ + /** + * @param args Command line args, if empty a help message is printed. + */ override def main(args: Array[String]): Unit = { val parser = new MahoutOptionParser[Options]("ItemSimilarity") { head("ItemSimilarity", "Spark") @@ -55,7 +57,8 @@ object ItemSimilarityDriver extends MahoutDriver { } opt[Int]("maxPrefs") abbr ("mppu") action { (x, options) => options.copy(maxPrefs = x) - } text ("Max number of preferences to consider per user or item, users or items with more preferences will be sampled down (optional). Default: 500") validate { x => + } text ("Max number of preferences to consider per user or item, users or items with more preferences will"+ + " be sampled down (optional). Default: 500") validate { x => if (x > 0) success else failure("Option --maxPrefs must be > 0") } opt[Int]("minPrefs") abbr ("mp") action { (x, options) => @@ -80,10 +83,12 @@ object ItemSimilarityDriver extends MahoutDriver { } opt[String]("filter1") abbr ("f1") action { (x, options) => options.copy(filter1 = x) - } text ("String whose presence indicates a datum for the primary item set, can be a regex (optional). Default: no filtered is applied, all is used") + } text ("String whose presence indicates a datum for the primary item set, can be a regex (optional)."+ + " Default: no filtered is applied, all is used") opt[String]("filter2") abbr ("f2") action { (x, options) => options.copy(filter2 = x) - } text ("String whose presence indicates a datum for the secondary item set, can be a regex (optional). Used in cross-cooccurrence. Default: no secondary filter is applied") + } text ("String whose presence indicates a datum for the secondary item set, can be a regex (optional)."+ + " Used in cross-cooccurrence. Default: no secondary filter is applied") opt[Int]("rowIDPosition") abbr ("rc") action { (x, options) => options.copy(rowIDPosition = x) } text ("Column number (0 based Int) containing the row ID string (optional). Default: 0") validate { x => @@ -103,17 +108,20 @@ object ItemSimilarityDriver extends MahoutDriver { note("\nFile input options:") opt[Unit]('r', "recursive") action { (_, options) => options.copy(recursive = true) - } text ("The input path should be searched recursively for files that match the filename pattern from -fp (optional), Default: false") + } text ("The input path should be searched recursively for files that match the filename pattern from -fp (optional),"+ + " Default: false") opt[String]("filenamePattern") abbr ("fp") action { (x, options) => options.copy(filenamePattern = x) - } text ("Regex to match in determining input files (optional). Default: filename in the --input option or '^part-.*' if --input is a directory") + } text ("Regex to match in determining input files (optional). Default: filename in the --input option or '^part-.*'"+ + " if --input is a directory") note("\nOutput text file schema options:") opt[String]("outDelim1") abbr ("od1") action { (x, options) => options.copy(outDelim1 = x) } text ("Primary output inDelim value, used to separate row IDs from the similar items list (optional). Default: '\\t'") opt[String]("outDelim2") abbr ("od2") action { (x, options) => options.copy(outDelim2 = x) - } text ("Secondary output inDelim value, used to separate item IDs from their values in the similar items list (optional). Default: ':'") + } text ("Secondary output inDelim value, used to separate item IDs from their values in the similar items list"+ + " (optional). Default: ':'") opt[String]("outDelim3") abbr ("od3") action { (x, options) => options.copy(outDelim3 = x) } text ("Last inDelim value, used to separate (itemID:value) tuples in the similar items list. (optional). Default: ','") @@ -123,44 +131,66 @@ object ItemSimilarityDriver extends MahoutDriver { checkConfig { c => if (c.filterPosition == c.itemIDPosition || c.filterPosition == c.rowIDPosition || c.rowIDPosition == c.itemIDPosition) failure("The row, item, and filter positions must be unique.") else success } + checkConfig { c => if (c.filter1 != null && c.filter2 != null && c.filter1 == c.filter2) failure("If using filters they must be unique.") else success } + } + parser.parse(args, Options()) map { opts => options = opts process } + } private def readIndexedDatasets: Array[IndexedDataset] = { - val inFiles = FileSysUtils(options.input, options.filenamePattern, options.recursive ).uris - if(inFiles.isEmpty){Array()}else{ + + val inFiles = FileSysUtils(options.input, options.filenamePattern, options.recursive).uris + + if (inFiles.isEmpty) { + Array() + } else { + val indexedDataset1 = IndexedDataset(readStore1.readFrom(inFiles)) + if (options.filterPosition != -1 && options.filter2 != null) { val indexedDataset2 = IndexedDataset(readStore2.readFrom(inFiles)) Array(indexedDataset1, indexedDataset2) } else { Array(indexedDataset1) } + } + } override def start(masterUrl: String = options.master, appName: String = options.appName, - customJars:Traversable[String] = Traversable.empty[String]): Unit = { - //todo: create and modify a SparkContext here, which can be passed in to mahoutSparkContext in the super.start - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "100") - System.setProperty("spark.executor.memory", "2g") + customJars: Traversable[String] = Traversable.empty[String]): Unit = { + + sparkConf.set("spark.kryo.referenceTracking", "false") + .set("spark.kryoserializer.buffer.mb", "100") + .set("spark.executor.memory", "2g") + super.start(masterUrl, appName, customJars) - val readSchema1 = new Schema("delim" -> options.inDelim, "filter" -> options.filter1, "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, "filterPosition" -> options.filterPosition) - if(options.filterPosition != -1 && options.filter2 != null) { - val readSchema2 = new Schema("delim" -> options.inDelim, "filter" -> options.filter2, "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, "filterPosition" -> options.filterPosition) - readStore2 = new TextDelimitedIndexedDatasetReader(readSchema2, mc) + //implicit val mc = this.mc + + val readSchema1 = new Schema("delim" -> options.inDelim, "filter" -> options.filter1, + "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, + "filterPosition" -> options.filterPosition) + readStore1 = new TextDelimitedIndexedDatasetReader(readSchema1) + + if (options.filterPosition != -1 && options.filter2 != null) { + val readSchema2 = new Schema("delim" -> options.inDelim, "filter" -> options.filter2, + "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, + "filterPosition" -> options.filterPosition) + readStore2 = new TextDelimitedIndexedDatasetReader(readSchema2) } - val writeSchema = new Schema("delim1" -> options.outDelim1, "delim2" -> options.outDelim2, "delim3" -> options.outDelim3) - readStore1 = new TextDelimitedIndexedDatasetReader(readSchema1, mc) - writeStore = new TextDelimitedIndexedDatasetWriter(writeSchema, mc) + + writeSchema = new Schema("delim1" -> options.outDelim1, "delim2" -> options.outDelim2, "delim3" -> options.outDelim3) + writeStore = new TextDelimitedIndexedDatasetWriter(writeSchema) + } override def process: Unit = { @@ -170,39 +200,47 @@ object ItemSimilarityDriver extends MahoutDriver { val indicatorMatrices = CooccurrenceAnalysis.cooccurrences(indexedDatasets(0).matrix, options.randomSeed, options.maxSimilaritiesPerItem, options.maxPrefs, Array(indexedDatasets(1).matrix)) - val selfIndicatorDataset = new IndexedDataset(indicatorMatrices(0), indexedDatasets(0).columnIDs, indexedDatasets(0).columnIDs) // self similarity + // self similarity + // the next two lines write the drm using a Writer class + // val selfIndicatorDataset = new IndexedDataset(indicatorMatrices(0), indexedDatasets(0).columnIDs, indexedDatasets(0).columnIDs) + // writeStore.writeTo(selfIndicatorDataset, options.output + "indicator-matrix") + + // an alternative is to create a version of IndexedDataset that knows how to write itself + val selfIndicatorDataset = new IndexedDatasetTextDelimitedWriteable(indicatorMatrices(0), indexedDatasets(0).columnIDs, + indexedDatasets(0).columnIDs, writeSchema) + selfIndicatorDataset.writeTo(options.output + "indicator-matrix") + + if (indexedDatasets.length > 1) { - writeStore.writeTo(selfIndicatorDataset, options.output+"indicator-matrix") - if(indexedDatasets.length > 1){ val crossIndicatorDataset = new IndexedDataset(indicatorMatrices(1), indexedDatasets(0).columnIDs, indexedDatasets(1).columnIDs) // cross similarity - writeStore.writeTo(crossIndicatorDataset, options.output+"cross-indicator-matrix") + writeStore.writeTo(crossIndicatorDataset, options.output + "cross-indicator-matrix") + } stop } - //Default values go here, any '_' or null should be 'required' in the Parser or flags an unused option + // Default values go here, any '_' or null should be 'required' in the Parser or flags an unused option // todo: support two input streams for cross-similarity, maybe assume one schema for both inputs case class Options( - master: String = "local", - appName: String = "ItemSimilarityJob", - randomSeed: Int = System.currentTimeMillis().toInt, - recursive: Boolean = false, - input: String = null, - output: String = null, - filenamePattern: String = "^part-.*", - maxSimilaritiesPerItem: Int = 100, - maxPrefs: Int = 500, - minPrefs: Int = 1, - rowIDPosition: Int = 0, - itemIDPosition: Int = 1, - filterPosition: Int = -1, - filter1: String = null, - filter2: String = null, - inDelim: String = ",", - outDelim1: String = "\t", - outDelim2: String = ":", - outDelim3: String = "," - ) + master: String = "local", + appName: String = "ItemSimilarityJob", + randomSeed: Int = System.currentTimeMillis().toInt, + recursive: Boolean = false, + input: String = null, + output: String = null, + filenamePattern: String = "^part-.*", + maxSimilaritiesPerItem: Int = 100, + maxPrefs: Int = 500, + minPrefs: Int = 1, + rowIDPosition: Int = 0, + itemIDPosition: Int = 1, + filterPosition: Int = -1, + filter1: String = null, + filter2: String = null, + inDelim: String = ",", + outDelim1: String = "\t", + outDelim2: String = ":", + outDelim3: String = ",") } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala index b8a79f8cac..6af7696ab5 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala @@ -17,10 +17,11 @@ package org.apache.mahout.drivers -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.mahout.sparkbindings._ -/** Extend this class to create a Mahout CLI driver. Minimally you must override process and main. Also define a command line parser and default options or fill in the following template: +/** Extend this class to create a Mahout CLI driver. Minimally you must override process and main. + * Also define a command line parser and default options or fill in the following template: * {{{ * object SomeDriver extends MahoutDriver { * override def main(args: Array[String]): Unit = { @@ -49,9 +50,12 @@ import org.apache.mahout.sparkbindings._ * }}} */ abstract class MahoutDriver { - protected var mc: SparkContext = _ + implicit var mc: SparkContext = _ + implicit val sparkConf = new SparkConf() + /** Creates a Spark context to run the job inside. - * Creates a Spark context to run the job inside. Override to set the SparkConf values specific to the job, these must be set before the context is created. + * Creates a Spark context to run the job inside. Override to set the SparkConf values specific to the job, + * these must be set before the context is created. * @param masterUrl Spark master URL * @param appName Name to display in Spark UI * @param customJars List of paths to custom jars @@ -69,6 +73,6 @@ abstract class MahoutDriver { /** This is wher you do the work, call start first, then before exiting call stop */ protected def process: Unit - /** Parse command line and call process*/ + /** Parse command line and call process */ def main(args: Array[String]): Unit } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala index af01ff0127..de17642bea 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -17,14 +17,12 @@ package org.apache.mahout.drivers -import org.apache.mahout.sparkbindings.drm._ import org.apache.spark.SparkContext._ import org.apache.mahout.math.RandomAccessSparseVector import org.apache.spark.SparkContext import com.google.common.collect.{BiMap, HashBiMap} import scala.collection.JavaConversions._ -import org.apache.mahout.sparkbindings.DrmRdd -import org.apache.mahout.math.drm.DrmLike +import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} import org.apache.mahout.sparkbindings._ @@ -33,20 +31,20 @@ import org.apache.mahout.sparkbindings._ * @todo the reader need not create both dictionaries but does at present. There are cases where one or the other dictionary is never used so saving the memory for a very large dictionary may be worth the optimization to specify which dictionaries are created. */ trait Reader[T]{ - val sc: SparkContext + val mc: SparkContext val readSchema: Schema - protected def reader(sc: SparkContext, readSchema: Schema, source: String): T - def readFrom(source: String): T = reader(sc, readSchema, source) + protected def reader(mc: SparkContext, readSchema: Schema, source: String): T + def readFrom(source: String): T = reader(mc, readSchema, source) } /** Writer trait is abstract in the sense that the writer method must be supplied by an extending trait, which also defines the type to be written. * @tparam T */ trait Writer[T]{ - val sc: SparkContext + val mc: SparkContext val writeSchema: Schema - protected def writer(sc: SparkContext, writeSchema: Schema, dest: String, collection: T): Unit - def writeTo(collection: T, dest: String) = writer(sc, writeSchema, dest, collection) + protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, collection: T): Unit + def writeTo(collection: T, dest: String) = writer(mc, writeSchema, dest, collection) } /** Extends Reader trait to supply the [[org.apache.mahout.drivers.IndexedDataset]] as the type read and a reader function for reading text delimited files as described in the [[org.apache.mahout.drivers.Schema]] @@ -54,12 +52,12 @@ trait Writer[T]{ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ /** Read in text delimited tuples from all URIs in this comma delimited source String. * - * @param sc context for the Spark job + * @param mc context for the Spark job * @param readSchema describes the delimiters and positions of values in the text delimited file. * @param source comma delimited URIs of text files to be read into the [[org.apache.mahout.drivers.IndexedDataset]] * @return */ - protected def reader(sc: SparkContext, readSchema: Schema, source: String): IndexedDataset = { + protected def reader(mc: SparkContext, readSchema: Schema, source: String): IndexedDataset = { try { val delimiter = readSchema("delim").asInstanceOf[String] val rowIDPosition = readSchema("rowIDPosition").asInstanceOf[Int] @@ -74,25 +72,25 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ throw new IllegalArgumentException }) - var columns = sc.textFile(source).map({ line => line.split(delimiter)}) + var columns = mc textFile(source) map({ line => line.split(delimiter)}) - columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) + columns = columns filter({ tokens => tokens(filterPosition) == filterBy}) - val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) + val interactions = columns map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) interactions.cache() - val rowIDs = interactions.map({ case (rowID, _) => rowID}).distinct().collect() - val columnIDs = interactions.map({ case (_, columnID) => columnID}).distinct().collect() + val rowIDs = interactions map({ case (rowID, _) => rowID}) distinct() collect() + val columnIDs = interactions map({ case (_, columnID) => columnID}) distinct() collect() val numRows = rowIDs.size val numColumns = columnIDs.size val rowIDDictionary = asOrderedDictionary(rowIDs) - val columnIDDictionary = asOrderedDictionary(columnIDs) + val rowIDDictionary_bcast = mc.broadcast(rowIDDictionary) - val rowIDDictionary_bcast = sc.broadcast(rowIDDictionary) - val columnIDDictionary_bcast = sc.broadcast(columnIDDictionary) + val columnIDDictionary = asOrderedDictionary(columnIDs) + val columnIDDictionary_bcast = mc.broadcast(columnIDDictionary) val indexedInteractions = interactions.map({ case (rowID, columnID) => @@ -108,8 +106,6 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ rowIndex -> row }).asInstanceOf[DrmRdd[Int]] - //todo: old API, val drmInteractions = new CheckpointedDrmBase[Int](indexedInteractions, numRows, numColumns) - //val drmInteractions = new CheckpointedDrmSpark[Int](indexedInteractions, numRows, numColumns) val drmInteractions = drmWrap[Int](indexedInteractions, numRows, numColumns) IndexedDataset(drmInteractions, rowIDDictionary, columnIDDictionary) @@ -135,11 +131,11 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ /** Read in text delimited tuples from all URIs in this comma delimited source String. * - * @param sc context for the Spark job + * @param mc context for the Spark job * @param writeSchema describes the delimiters and positions of values in the output text delimited file. * @param dest directory to write text delimited version of [[org.apache.mahout.drivers.IndexedDataset]] */ - protected def writer(sc: SparkContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { + protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { try { val outDelim1 = writeSchema("delim1").asInstanceOf[String] val outDelim2 = writeSchema("delim2").asInstanceOf[String] @@ -174,21 +170,53 @@ trait TDIndexedDatasetReaderWriter extends TDIndexedDatasetReader with TDIndexed /** Reads text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. * @param readSchema describes the delimiters and position of values in the text delimited file to be read. - * @param sc Spark context for reading files - * @note The source files are supplied to the readFrom trait method. + * @param mc Spark context for reading files + * @note The source is supplied by Reader#readFrom . * */ -class TextDelimitedIndexedDatasetReader(val readSchema: Schema, val sc: SparkContext) extends TDIndexedDatasetReader +class TextDelimitedIndexedDatasetReader(val readSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReader /** Writes text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. - * @param sc Spark context for reading files - * @note the destination if supplied to the writeTo trait method + * @param mc Spark context for reading files + * @note the destination is supplied by Writer#writeTo trait method * */ -class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema, val sc: SparkContext) extends TDIndexedDatasetWriter +class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetWriter /** Reads and writes text delimited files to/from an IndexedDataset. Classes are needed to supply trait params in their constructor. * @param readSchema describes the delimiters and position of values in the text delimited file(s) to be read. * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. - * @param sc Spark context for reading the files + * @param mc Spark context for reading the files, may be implicitly defined. + * */ +class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReaderWriter + +/** A version of IndexedDataset that has it's own writeTo method from a Writer trait. This is an alternative to creating + * a Writer based stand-alone class for writing. Consider it experimental allowing similar semantics to drm.writeDrm(). + * Experimental because it's not clear that it is simpler or more intuitive and since IndexedDatasetTextDelimitedWriteables + * are probably short lived in terms of lines of code so complexity may be moot. + * @param matrix the data + * @param rowIDs bi-directional dictionary for rows of external IDs to internal ordinal Mahout IDs. + * @param columnIDs bi-directional dictionary for columns of external IDs to internal ordinal Mahout IDs. + * @param writeSchema contains params for the schema/format or the written text delimited file. + * @param mc mahout distributed context (SparkContext) may be implicitly defined. * */ -class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema, val sc: SparkContext) extends TDIndexedDatasetReaderWriter +class IndexedDatasetTextDelimitedWriteable(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int], + val writeSchema: Schema)(implicit val mc: SparkContext) + extends IndexedDataset(matrix, rowIDs, columnIDs) with TDIndexedDatasetWriter { + + def writeTo(dest: String): Unit = { + writeTo(this, dest) + } +} + +/** + * Companion object for the case class [[org.apache.mahout.drivers.IndexedDatasetTextDelimitedWriteable]] primarily used to get a secondary constructor for + * making one [[org.apache.mahout.drivers.IndexedDatasetTextDelimitedWriteable]] from another. Used when you have a factory like [[org.apache.mahout.drivers.IndexedDatasetStore]] + * {{{ + * val id = IndexedDatasetTextDelimitedWriteable(indexedDatasetStore.read) + * }}} + */ + +object IndexedDatasetTextDelimitedWriteable { + /** Secondary constructor for [[org.apache.mahout.drivers.IndexedDataset]] */ + def apply(id2: IndexedDatasetTextDelimitedWriteable) = new IndexedDatasetTextDelimitedWriteable(id2.matrix, id2.rowIDs, id2.columnIDs, id2.writeSchema)(id2.mc) +} diff --git a/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala b/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala index 13fa292b92..50302a6bfa 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -25,6 +24,6 @@ import scala.collection.mutable.HashMap * @param params list of mappings for instantiation {{{val mySchema = new Schema("one" -> 1, "two" -> "2", ...)}}} */ class Schema(params: Tuple2[String, Any]*) extends HashMap[String, Any] { - //todo: this require a mutable HashMap, do we care? + // todo: this require a mutable HashMap, do we care? this ++= params } diff --git a/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala b/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala index fb13361090..2b2a960567 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala @@ -1,7 +1,3 @@ -package org.apache.mahout.drivers - -import org.scalatest.FunSuite - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -19,7 +15,11 @@ import org.scalatest.FunSuite * limitations under the License. */ +package org.apache.mahout.drivers + +import org.scalatest.FunSuite + class IndexedDatasetTest extends FunSuite { - //todo: put some tests here! + // todo: put some tests here! } diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala index c1ae4ab5a3..b98d48f6fd 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala @@ -1,12 +1,3 @@ -package org.apache.mahout.drivers - -import org.scalatest.{BeforeAndAfter, FunSuite} - -import org.apache.mahout.sparkbindings._ -import java.io.{FileWriter, BufferedWriter} -import com.google.common.io.Closeables -import org.apache.spark.SparkContext - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -24,117 +15,82 @@ import org.apache.spark.SparkContext * limitations under the License. */ -class ItemSimilarityDriver$Test extends FunSuite { - private var sc: SparkContext = _ +package org.apache.mahout.drivers +import org.scalatest.FunSuite + +class ItemSimilarityDriver$Test extends FunSuite { test ("Running some text delimited files through the import/cooccurrence/export"){ val exampleCsvlogStatements = Array( - "12569537329,user1,item1,\"view\"", - "12569537329,user1,item2,\"view\"", - "12569537329,user1,item2,\"like\"", - "12569537329,user1,item3,\"like\"", - "12569537329,user2,item2,\"view\"", - "12569537329,user2,item2,\"like\"", - "12569537329,user3,item1,\"like\"", - "12569537329,user3,item3,\"view\"", - "12569537329,user3,item3,\"like\"" - ) + "12569537329,user1,item1,\"view\"", + "12569537329,user1,item2,\"view\"", + "12569537329,user1,item2,\"like\"", + "12569537329,user1,item3,\"like\"", + "12569537329,user2,item2,\"view\"", + "12569537329,user2,item2,\"like\"", + "12569537329,user3,item1,\"like\"", + "12569537329,user3,item3,\"view\"", + "12569537329,user3,item3,\"like\"") val exampleTsvLogStatements = Array( - "12569537329\tuser1\titem1\t\"view\"", - "12569537329\tuser1\titem2\t\"view\"", - "12569537329\tuser1\titem2\t\"like\"", - "12569537329\tuser1\titem3\t\"like\"", - "12569537329\tuser2\titem2\t\"view\"", - "12569537329\tuser2\titem2\t\"like\"", - "12569537329\tuser3\titem1\t\"like\"", - "12569537329\tuser3\titem3\t\"view\"", - "12569537329\tuser3\titem3\t\"like\"" - ) + "12569537329\tuser1\titem1\t\"view\"", + "12569537329\tuser1\titem2\t\"view\"", + "12569537329\tuser1\titem2\t\"like\"", + "12569537329\tuser1\titem3\t\"like\"", + "12569537329\tuser2\titem2\t\"view\"", + "12569537329\tuser2\titem2\t\"like\"", + "12569537329\tuser3\titem1\t\"like\"", + "12569537329\tuser3\titem3\t\"view\"", + "12569537329\tuser3\titem3\t\"like\"") val csvLogStatements = Array( - "u1,purchase,iphone", - "u1,purchase,ipad", - "u2,purchase,nexus", - "u2,purchase,galaxy", - "u3,purchase,surface", - "u4,purchase,iphone", - "u4,purchase,galaxy", - "u1,view,iphone", - "u1,view,ipad", - "u1,view,nexus", - "u1,view,galaxy", - "u2,view,iphone", - "u2,view,ipad", - "u2,view,nexus", - "u2,view,galaxy", - "u3,view,surface", - "u3,view,nexus", - "u4,view,iphone", - "u4,view,ipad", - "u4,view,galaxy" - ) + "u1,purchase,iphone", + "u1,purchase,ipad", + "u2,purchase,nexus", + "u2,purchase,galaxy", + "u3,purchase,surface", + "u4,purchase,iphone", + "u4,purchase,galaxy", + "u1,view,iphone", + "u1,view,ipad", + "u1,view,nexus", + "u1,view,galaxy", + "u2,view,iphone", + "u2,view,ipad", + "u2,view,nexus", + "u2,view,galaxy", + "u3,view,surface", + "u3,view,nexus", + "u4,view,iphone", + "u4,view,ipad", + "u4,view,galaxy") val tsvLogStatements = Array( - "u1\tpurchase\tiphone", - "u1\tpurchase\tipad", - "u2\tpurchase\tnexus", - "u2\tpurchase\tgalaxy", - "u3\tpurchase\tsurface", - "u4\tpurchase\tiphone", - "u4\tpurchase\tgalaxy", - "u1\tview\tiphone", - "u1\tview\tipad", - "u1\tview\tnexus", - "u1\tview\tgalaxy", - "u2\tview\tiphone", - "u2\tview\tipad", - "u2\tview\tnexus", - "u2\tview\tgalaxy", - "u3\tview\tsurface", - "u3\tview\tnexus", - "u4\tview\tiphone", - "u4\tview\tipad", - "u4\tview\tgalaxy" - ) - - var w: BufferedWriter = null - //try { - w = new BufferedWriter(new FileWriter("/tmp/cf-data.txt")) - w.write(csvLogStatements.mkString("\n")) - //} finally { - Closeables.close(w, false) - //} - /* - val indexedLikes = IndexedDatasetStore.readTuples(sc, "tmp/cf-data.txt", 2, ",", 0, 2, 1, "purchase") - - val indexedViews = IndexedDatasetStore.readTuples(sc, "/tmp/cf-data.txt", 2, ",", 0, 2, 1, "view") - - val drmLikes = indexedLikes.matrix - val drmViews = indexedViews.matrix - - // Now we could run cooccurrence analysis using the DRMs, instead we'll just fetch and print the matrices - val drmXCooccurrences = cooccurrences(drmLikes, randomSeed = 0xdeadbeef, - maxInterestingItemsPerThing = 100, maxNumInteractions = 500, Array(drmViews)) - - val inCoreViews = drmViews.collect - val inCoreLikes = drmLikes.collect - val inCoreIndicator = drmXCooccurrences(0).collect - val inCoreXIndicator = drmXCooccurrences(1).collect - println("\nLIKES:") - println(inCoreLikes) - println("\nVIEWS:") - println(inCoreViews) - println("\nINDICATOR MATRIX") - println(inCoreIndicator) - println("\nCROSS INDICATOR MATRIX") - println(inCoreXIndicator) - */ - - /* + "u1\tpurchase\tiphone", + "u1\tpurchase\tipad", + "u2\tpurchase\tnexus", + "u2\tpurchase\tgalaxy", + "u3\tpurchase\tsurface", + "u4\tpurchase\tiphone", + "u4\tpurchase\tgalaxy", + "u1\tview\tiphone", + "u1\tview\tipad", + "u1\tview\tnexus", + "u1\tview\tgalaxy", + "u2\tview\tiphone", + "u2\tview\tipad", + "u2\tview\tnexus", + "u2\tview\tgalaxy", + "u3\tview\tsurface", + "u3\tview\tnexus", + "u4\tview\tiphone", + "u4\tview\tipad", + "u4\tview\tgalaxy") + + /* //Clustered Spark and HDFS ItemSimilarityDriver.main(Array( "--input", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/cf-data.txt", @@ -148,7 +104,9 @@ class ItemSimilarityDriver$Test extends FunSuite { "--filterPosition", "1" )) */ - //local multi-threaded Spark with HDFS using large dataset + // local multi-threaded Spark with HDFS using large dataset + // todo: not sure how to handle build testing on HDFS maybe make into an integration test + // or example. /* ItemSimilarityDriver.main(Array( "--input", "hdfs://occam4:54310/user/pat/xrsj/ratings_data.txt", "--output", "hdfs://occam4:54310/user/pat/xrsj/indicatorMatrices/", @@ -162,10 +120,12 @@ class ItemSimilarityDriver$Test extends FunSuite { )) */ - //local multi-threaded Spark with local FS + // local multi-threaded Spark with local FS, suitable for build tests but need better location for data + // todo: remove absolute path + // todo: check computed value or it's not much of a test. ItemSimilarityDriver.main(Array( - "--input", "/tmp/cf-data.txt", - "--output", "tmp/indicatorMatrices/", + "--input", "/Users/pat/big-data/tmp/cf-data.txt", + "--output", "/Users/pat/big-data/tmp/indicatorMatrices/", "--master", "local[4]", "--filter1", "purchase", "--filter2", "view", From 996ccfb82a8ed3ff90f51968e661b2449f3c4759 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 19 Jun 2014 10:46:23 -0700 Subject: [PATCH 09/38] for high level review, not ready for merge. changed to dot notation --- .../scala/org/apache/mahout/drivers/ReaderWriter.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala index de17642bea..0f4b3c0b70 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -72,16 +72,16 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ throw new IllegalArgumentException }) - var columns = mc textFile(source) map({ line => line.split(delimiter)}) + var columns = mc.textFile(source).map({ line => line.split(delimiter)}) - columns = columns filter({ tokens => tokens(filterPosition) == filterBy}) + columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) - val interactions = columns map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) + val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) interactions.cache() - val rowIDs = interactions map({ case (rowID, _) => rowID}) distinct() collect() - val columnIDs = interactions map({ case (_, columnID) => columnID}) distinct() collect() + val rowIDs = interactions.map({ case (rowID, _) => rowID}).distinct().collect() + val columnIDs = interactions.map({ case (_, columnID) => columnID}).distinct().collect() val numRows = rowIDs.size val numColumns = columnIDs.size From f62ab071869ee205ad398a3e094d871138e11a9e Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 19 Jun 2014 11:13:44 -0700 Subject: [PATCH 10/38] for high level review, not ready for merge. fixed a couple scaladoc refs --- .../main/scala/org/apache/mahout/drivers/IndexedDataset.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala index bd473daec1..0903750e61 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala @@ -46,7 +46,7 @@ case class IndexedDataset(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int * Companion object for the case class [[org.apache.mahout.drivers.IndexedDataset]] primarily used to get a secondary constructor for * making one [[org.apache.mahout.drivers.IndexedDataset]] from another. Used when you have a factory like [[org.apache.mahout.drivers.IndexedDatasetStore]] * {{{ - * val indexedDataset = IndexedDataset(indexedDatasetStore.read) + * val indexedDataset = IndexedDataset(indexedDatasetReader.readFrom(source)) * }}} */ From cbef0ee6264c28d0597cb2507427a647771c9bcd Mon Sep 17 00:00:00 2001 From: pferrel Date: Mon, 23 Jun 2014 14:49:20 -0700 Subject: [PATCH 11/38] adding tests, had to modify some test framework Scala to make the masterUrl visible to tests --- .../mahout/drivers/ItemSimilarityDriver.scala | 182 +++++++++------ .../apache/mahout/drivers/ReaderWriter.scala | 174 -------------- .../drivers/TextDelimitedReaderWriter.scala | 217 ++++++++++++++++++ .../drivers/ItemSimilarityDriver$Test.scala | 140 ----------- .../drivers/ItemSimilarityDriverSuite.scala | 174 ++++++++++++++ .../test/MahoutLocalContext.scala | 4 +- 6 files changed, 508 insertions(+), 383 deletions(-) create mode 100644 spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala delete mode 100644 spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala create mode 100644 spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 4b86f42dbf..faf69496d5 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -21,21 +21,26 @@ import org.apache.mahout.cf.CooccurrenceAnalysis /** * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. - * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. It reads text lines - * that contain (row id, column id, ...). The IDs are user specified strings which will be preserved in the + * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. + * It reads text lines + * that contain (row id, column id, ...). The IDs are user specified strings which will be + * preserved in the * output. The individual tuples will be accumulated into a matrix and [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]] * will be used to calculate row-wise self-similarity, or when using filters, will generate two - * matrices and calculate both the self similarity of the primary matrix and the row-wise similarity of the primary - * to the secondary. Returns one or two directories of text files formatted as specified in the options. + * matrices and calculate both the self similarity of the primary matrix and the row-wise + * similarity of the primary + * to the secondary. Returns one or two directories of text files formatted as specified in + * the options. */ object ItemSimilarityDriver extends MahoutDriver { //todo: Should also take two input streams and do cross similarity with no filter required. - //todo: Required: should work on the old Mahout TDF that allows values (but ignores them), required for examples + //todo: Required: should work on the old Mahout TDF that allows values (but ignores them), + // required for examples private var options: Options = _ - private var readStore1: TextDelimitedIndexedDatasetReader = _ - private var readStore2: TextDelimitedIndexedDatasetReader = _ - private var writeStore: TextDelimitedIndexedDatasetWriter = _ + private var reader1: TextDelimitedIndexedDatasetReader = _ + private var reader2: TextDelimitedIndexedDatasetReader = _ + private var writer: TextDelimitedIndexedDatasetWriter = _ private var writeSchema: Schema = _ /** @@ -44,23 +49,32 @@ object ItemSimilarityDriver extends MahoutDriver { override def main(args: Array[String]): Unit = { val parser = new MahoutOptionParser[Options]("ItemSimilarity") { head("ItemSimilarity", "Spark") + + //Input output options, non-driver specific note("Input, output options") opt[String]('i', "input") required() action { (x, options) => options.copy(input = x) } text ("Path for input. It may be a filename or directory name and can be a local file path or an HDFS URI (required).") + opt[String]('o', "output") required() action { (x, options) => - options.copy(output = x) + if (x.endsWith("/")) // todo: check to see if HDFS allows MS-Windows backslashes locally? + options.copy(output = x) + else + options.copy(output = x + "/") } text ("Output will be in sub-directories stored here so this must be a directory path (required).") + + //Algorithm control options--driver specific note("\nAlgorithm control options:") opt[String]("master") abbr ("ma") text ("URL for the Spark Master. (optional). Default: 'local'") action { (x, options) => options.copy(master = x) } + opt[Int]("maxPrefs") abbr ("mppu") action { (x, options) => options.copy(maxPrefs = x) - } text ("Max number of preferences to consider per user or item, users or items with more preferences will"+ - " be sampled down (optional). Default: 500") validate { x => + } text ("Max number of preferences to consider per user or item, users or items with more preferences will be sampled down (optional). Default: 500") validate { x => if (x > 0) success else failure("Option --maxPrefs must be > 0") } + opt[Int]("minPrefs") abbr ("mp") action { (x, options) => options.copy(minPrefs = x) } text ("Ignore users with less preferences than this (optional). Default: 1") validate { x => @@ -72,72 +86,93 @@ object ItemSimilarityDriver extends MahoutDriver { } text ("Try to cap the number of similar items for each item to this number (optional). Default: 100") validate { x => if (x > 0) success else failure("Option --maxSimilaritiesPerItem must be > 0") } + opt[Int]("randomSeed") abbr ("rs") action { (x, options) => options.copy(randomSeed = x) } text ("Int to seed random number generator (optional). Default: Uses time to generate a seed") validate { x => if (x > 0) success else failure("Option --randomSeed must be > 0") } + + //Input text file schema--not driver specific but input data specific, tuples input, + // not drms note("\nInput text file schema options:") opt[String]("inDelim") abbr ("d") text ("Input delimiter character (optional). Default: '\\t'") action { (x, options) => options.copy(inDelim = x) } + opt[String]("filter1") abbr ("f1") action { (x, options) => options.copy(filter1 = x) - } text ("String whose presence indicates a datum for the primary item set, can be a regex (optional)."+ - " Default: no filtered is applied, all is used") + } text ("String whose presence indicates a datum for the primary item set, can be a regex (optional). Default: no filtered is applied, all is used") + opt[String]("filter2") abbr ("f2") action { (x, options) => options.copy(filter2 = x) - } text ("String whose presence indicates a datum for the secondary item set, can be a regex (optional)."+ - " Used in cross-cooccurrence. Default: no secondary filter is applied") + } text ("String whose presence indicates a datum for the secondary item set, can be a regex (optional). Used in cross-cooccurrence. Default: no secondary filter is applied") + opt[Int]("rowIDPosition") abbr ("rc") action { (x, options) => options.copy(rowIDPosition = x) } text ("Column number (0 based Int) containing the row ID string (optional). Default: 0") validate { x => if (x >= 0) success else failure("Option --rowIDColNum must be >= 0") } + opt[Int]("itemIDPosition") abbr ("ic") action { (x, options) => options.copy(itemIDPosition = x) } text ("Column number (0 based Int) containing the item ID string (optional). Default: 1") validate { x => if (x >= 0) success else failure("Option --itemIDColNum must be >= 0") } + opt[Int]("filterPosition") abbr ("fc") action { (x, options) => options.copy(filterPosition = x) } text ("Column number (0 based Int) containing the filter string (optional). Default: -1 for no filter") validate { x => if (x >= -1) success else failure("Option --filterColNum must be >= -1") } + note("\nDefault input schema will accept: 'userIDitemId' or 'userIDitemIDany-text...' and all rows will be used") + + //File finding strategy--not driver specific note("\nFile input options:") opt[Unit]('r', "recursive") action { (_, options) => options.copy(recursive = true) - } text ("The input path should be searched recursively for files that match the filename pattern from -fp (optional),"+ - " Default: false") + } text ("The input path should be searched recursively for files that match the filename pattern from -fp (optional), Default: false") + opt[String]("filenamePattern") abbr ("fp") action { (x, options) => options.copy(filenamePattern = x) - } text ("Regex to match in determining input files (optional). Default: filename in the --input option or '^part-.*'"+ - " if --input is a directory") + } text ("Regex to match in determining input files (optional). Default: filename in the --input option or '^part-.*' if --input is a directory") + + //Drm output schema--not driver specific, drm specific note("\nOutput text file schema options:") - opt[String]("outDelim1") abbr ("od1") action { (x, options) => - options.copy(outDelim1 = x) - } text ("Primary output inDelim value, used to separate row IDs from the similar items list (optional). Default: '\\t'") - opt[String]("outDelim2") abbr ("od2") action { (x, options) => - options.copy(outDelim2 = x) - } text ("Secondary output inDelim value, used to separate item IDs from their values in the similar items list"+ - " (optional). Default: ':'") - opt[String]("outDelim3") abbr ("od3") action { (x, options) => - options.copy(outDelim3 = x) - } text ("Last inDelim value, used to separate (itemID:value) tuples in the similar items list. (optional). Default: ','") - note("\nDefault delimiters will produce output of the form: 'itemID1>itemID2:value2,itemID10:value10...'") + opt[String]("rowKeyDelim") abbr ("rd") action { (x, options) => + options.copy(rowKeyDelim = x) + } text ("Separates the rowID key from the vector values list (optional). Default: '\\t'") + + opt[String]("columnIdStrengthDelim") abbr ("cd") action { (x, options) => + options.copy(columnIdStrengthDelim = x) + } text ("Separates column IDs from their values in the vector values list (optional). Default: ':'") + + opt[String]("tupleDelim") abbr ("td") action { (x, options) => + options.copy(tupleDelim = x) + } text ("Separates vector tuple values in the values list (optional). Default: ','") + + note("\nDefault delimiters will produce output of the form: 'itemID1itemID2:value2,itemID10:value10...'") + + //Driver notes--driver specific note("\nNote: Only the Log Likelihood Ratio (LLR) is supported as a similarity measure.\n") + help("help") abbr ("h") text ("prints this usage text\n") + checkConfig { c => - if (c.filterPosition == c.itemIDPosition || c.filterPosition == c.rowIDPosition || c.rowIDPosition == c.itemIDPosition) failure("The row, item, and filter positions must be unique.") else success + if (c.filterPosition == c.itemIDPosition || c.filterPosition == c.rowIDPosition || c.rowIDPosition == c.itemIDPosition) failure("The row, item, and filter positions must be" + + " unique.") else success } + //check for option consistency, probably driver specific checkConfig { c => - if (c.filter1 != null && c.filter2 != null && c.filter1 == c.filter2) failure("If using filters they must be unique.") else success + if (c.filter1 != null && c.filter2 != null && c.filter1 == c.filter2) failure("If" + + " using filters they must be unique.") else success } } + //repeated code, should this be put base MahoutDriver somehow? parser.parse(args, Options()) map { opts => options = opts process @@ -145,51 +180,61 @@ object ItemSimilarityDriver extends MahoutDriver { } - private def readIndexedDatasets: Array[IndexedDataset] = { - - val inFiles = FileSysUtils(options.input, options.filenamePattern, options.recursive).uris - - if (inFiles.isEmpty) { - Array() - } else { - - val indexedDataset1 = IndexedDataset(readStore1.readFrom(inFiles)) - - if (options.filterPosition != -1 && options.filter2 != null) { - val indexedDataset2 = IndexedDataset(readStore2.readFrom(inFiles)) - Array(indexedDataset1, indexedDataset2) - } else { - Array(indexedDataset1) - } - - } - - } - - override def start(masterUrl: String = options.master, appName: String = options.appName, - customJars: Traversable[String] = Traversable.empty[String]): Unit = { + override def start(masterUrl: String = options.master, + appName: String = options.appName, + customJars:Traversable[String] = Traversable.empty[String]): + Unit = { sparkConf.set("spark.kryo.referenceTracking", "false") .set("spark.kryoserializer.buffer.mb", "100") .set("spark.executor.memory", "2g") - super.start(masterUrl, appName, customJars) - //implicit val mc = this.mc + super.start(masterUrl, appName) val readSchema1 = new Schema("delim" -> options.inDelim, "filter" -> options.filter1, - "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, + "rowIDPosition" -> options.rowIDPosition, + "columnIDPosition" -> options.itemIDPosition, "filterPosition" -> options.filterPosition) - readStore1 = new TextDelimitedIndexedDatasetReader(readSchema1) + + reader1 = new TextDelimitedIndexedDatasetReader(readSchema1) if (options.filterPosition != -1 && options.filter2 != null) { val readSchema2 = new Schema("delim" -> options.inDelim, "filter" -> options.filter2, - "rowIDPosition" -> options.rowIDPosition, "columnIDPosition" -> options.itemIDPosition, + "rowIDPosition" -> options.rowIDPosition, + "columnIDPosition" -> options.itemIDPosition, "filterPosition" -> options.filterPosition) - readStore2 = new TextDelimitedIndexedDatasetReader(readSchema2) + + reader2 = new TextDelimitedIndexedDatasetReader(readSchema2) } - writeSchema = new Schema("delim1" -> options.outDelim1, "delim2" -> options.outDelim2, "delim3" -> options.outDelim3) - writeStore = new TextDelimitedIndexedDatasetWriter(writeSchema) + writeSchema = new Schema( + "rowKeyDelim" -> options.rowKeyDelim, + "columnIdStrengthDelim" -> options.columnIdStrengthDelim, + "tupleDelim" -> options.tupleDelim) + + writer = new TextDelimitedIndexedDatasetWriter(writeSchema) + + } + + private def readIndexedDatasets: Array[IndexedDataset] = { + + val inFiles = FileSysUtils(options.input, options.filenamePattern, options.recursive).uris + + if (inFiles.isEmpty) { + Array() + } else { + + val selfSimilarityDataset = IndexedDataset(reader1.readFrom(inFiles)) + + if (options.filterPosition != -1 && options.filter2 != null) { + // todo: needs to support more than one cross-similarity indicator + val crossSimilarityDataset1 = IndexedDataset(reader2.readFrom(inFiles)) + Array(selfSimilarityDataset, crossSimilarityDataset1) + } else { + Array(selfSimilarityDataset) + } + + } } @@ -210,10 +255,11 @@ object ItemSimilarityDriver extends MahoutDriver { indexedDatasets(0).columnIDs, writeSchema) selfIndicatorDataset.writeTo(options.output + "indicator-matrix") + // todo: needs to support more than one cross-similarity indicator if (indexedDatasets.length > 1) { val crossIndicatorDataset = new IndexedDataset(indicatorMatrices(1), indexedDatasets(0).columnIDs, indexedDatasets(1).columnIDs) // cross similarity - writeStore.writeTo(crossIndicatorDataset, options.output + "cross-indicator-matrix") + writer.writeTo(crossIndicatorDataset, options.output + "cross-indicator-matrix") } @@ -221,7 +267,7 @@ object ItemSimilarityDriver extends MahoutDriver { } // Default values go here, any '_' or null should be 'required' in the Parser or flags an unused option - // todo: support two input streams for cross-similarity, maybe assume one schema for both inputs + // todo: support two input streams for cross-similarity, maybe assume one schema for all inputs case class Options( master: String = "local", appName: String = "ItemSimilarityJob", @@ -239,8 +285,8 @@ object ItemSimilarityDriver extends MahoutDriver { filter1: String = null, filter2: String = null, inDelim: String = ",", - outDelim1: String = "\t", - outDelim2: String = ":", - outDelim3: String = ",") + rowKeyDelim: String = "\t", + columnIdStrengthDelim: String = ":", + tupleDelim: String = ",") } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala index 0f4b3c0b70..ae0fea8d7b 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -46,177 +46,3 @@ trait Writer[T]{ protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, collection: T): Unit def writeTo(collection: T, dest: String) = writer(mc, writeSchema, dest, collection) } - -/** Extends Reader trait to supply the [[org.apache.mahout.drivers.IndexedDataset]] as the type read and a reader function for reading text delimited files as described in the [[org.apache.mahout.drivers.Schema]] - */ -trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ - /** Read in text delimited tuples from all URIs in this comma delimited source String. - * - * @param mc context for the Spark job - * @param readSchema describes the delimiters and positions of values in the text delimited file. - * @param source comma delimited URIs of text files to be read into the [[org.apache.mahout.drivers.IndexedDataset]] - * @return - */ - protected def reader(mc: SparkContext, readSchema: Schema, source: String): IndexedDataset = { - try { - val delimiter = readSchema("delim").asInstanceOf[String] - val rowIDPosition = readSchema("rowIDPosition").asInstanceOf[Int] - val columnIDPosition = readSchema("columnIDPosition").asInstanceOf[Int] - val filterPosition = readSchema("filterPosition").asInstanceOf[Int] - val filterBy = readSchema("filter").asInstanceOf[String] - //instance vars must be put into locally scoped vals when used in closures that are - //executed but Spark - - assert(!source.isEmpty, { - println(this.getClass.toString + ": has no files to read") - throw new IllegalArgumentException - }) - - var columns = mc.textFile(source).map({ line => line.split(delimiter)}) - - columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) - - val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) - - interactions.cache() - - val rowIDs = interactions.map({ case (rowID, _) => rowID}).distinct().collect() - val columnIDs = interactions.map({ case (_, columnID) => columnID}).distinct().collect() - - val numRows = rowIDs.size - val numColumns = columnIDs.size - - val rowIDDictionary = asOrderedDictionary(rowIDs) - val rowIDDictionary_bcast = mc.broadcast(rowIDDictionary) - - val columnIDDictionary = asOrderedDictionary(columnIDs) - val columnIDDictionary_bcast = mc.broadcast(columnIDDictionary) - - val indexedInteractions = - interactions.map({ case (rowID, columnID) => - val rowIndex = rowIDDictionary_bcast.value.get(rowID).get - val columnIndex = columnIDDictionary_bcast.value.get(columnID).get - - rowIndex -> columnIndex - }).groupByKey().map({ case (rowIndex, columnIndexes) => - val row = new RandomAccessSparseVector(numColumns) - for (columnIndex <- columnIndexes) { - row.setQuick(columnIndex, 1.0) - } - rowIndex -> row - }).asInstanceOf[DrmRdd[Int]] - - val drmInteractions = drmWrap[Int](indexedInteractions, numRows, numColumns) - - IndexedDataset(drmInteractions, rowIDDictionary, columnIDDictionary) - - } catch { - case cce: ClassCastException => { - println(this.getClass.toString + ": Schema has illegal values"); throw cce - } - } - } - - private def asOrderedDictionary(entries: Array[String]): BiMap[String, Int] = { - var dictionary: BiMap[String, Int] = HashBiMap.create() - var index = 0 - for (entry <- entries) { - dictionary.forcePut(entry, index) - index += 1 - } - dictionary - } -} - -trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ - /** Read in text delimited tuples from all URIs in this comma delimited source String. - * - * @param mc context for the Spark job - * @param writeSchema describes the delimiters and positions of values in the output text delimited file. - * @param dest directory to write text delimited version of [[org.apache.mahout.drivers.IndexedDataset]] - */ - protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { - try { - val outDelim1 = writeSchema("delim1").asInstanceOf[String] - val outDelim2 = writeSchema("delim2").asInstanceOf[String] - val outDelim3 = writeSchema("delim3").asInstanceOf[String] - //instance vars must be put into locally scoped vals when put into closures that are - //executed but Spark - assert (indexedDataset != null, {println(this.getClass.toString+": has no indexedDataset to write"); throw new IllegalArgumentException }) - assert (!dest.isEmpty, {println(this.getClass.toString+": has no destination or indextedDataset to write"); throw new IllegalArgumentException}) - val matrix: DrmLike[Int] = indexedDataset.matrix - val rowIDDictionary: BiMap[String, Int] = indexedDataset.rowIDs - val columnIDDictionary: BiMap[String, Int] = indexedDataset.columnIDs - // below doesn't compile because the rdd is not in a CheckpointedDrmSpark also I don't know how to turn a - // CheckpointedDrmSpark[Int] into a DrmLike[Int], which I need to pass in the CooccurrenceAnalysis#cooccurrence - // This seems to be about the refacotring to abstract away from Spark but the Read and Write are Spark specific - // and the non-specific DrmLike is no longer attached to a CheckpointedDrmSpark, could be missing something though - matrix.rdd.map({ case (rowID, itemVector) => - var line: String = rowIDDictionary.inverse.get(rowID) + outDelim1 - for (item <- itemVector.nonZeroes()) { - line += columnIDDictionary.inverse.get(item.index) + outDelim2 + item.get + outDelim3 - } - line.dropRight(1) - }) - .saveAsTextFile(dest) - }catch{ - case cce: ClassCastException => {println(this.getClass.toString+": Schema has illegal values"); throw cce} - } - } -} - -/** A combined trait that reads and writes */ -trait TDIndexedDatasetReaderWriter extends TDIndexedDatasetReader with TDIndexedDatasetWriter - -/** Reads text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. - * @param readSchema describes the delimiters and position of values in the text delimited file to be read. - * @param mc Spark context for reading files - * @note The source is supplied by Reader#readFrom . - * */ -class TextDelimitedIndexedDatasetReader(val readSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReader - -/** Writes text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. - * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. - * @param mc Spark context for reading files - * @note the destination is supplied by Writer#writeTo trait method - * */ -class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetWriter - -/** Reads and writes text delimited files to/from an IndexedDataset. Classes are needed to supply trait params in their constructor. - * @param readSchema describes the delimiters and position of values in the text delimited file(s) to be read. - * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. - * @param mc Spark context for reading the files, may be implicitly defined. - * */ -class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReaderWriter - -/** A version of IndexedDataset that has it's own writeTo method from a Writer trait. This is an alternative to creating - * a Writer based stand-alone class for writing. Consider it experimental allowing similar semantics to drm.writeDrm(). - * Experimental because it's not clear that it is simpler or more intuitive and since IndexedDatasetTextDelimitedWriteables - * are probably short lived in terms of lines of code so complexity may be moot. - * @param matrix the data - * @param rowIDs bi-directional dictionary for rows of external IDs to internal ordinal Mahout IDs. - * @param columnIDs bi-directional dictionary for columns of external IDs to internal ordinal Mahout IDs. - * @param writeSchema contains params for the schema/format or the written text delimited file. - * @param mc mahout distributed context (SparkContext) may be implicitly defined. - * */ -class IndexedDatasetTextDelimitedWriteable(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int], - val writeSchema: Schema)(implicit val mc: SparkContext) - extends IndexedDataset(matrix, rowIDs, columnIDs) with TDIndexedDatasetWriter { - - def writeTo(dest: String): Unit = { - writeTo(this, dest) - } -} - -/** - * Companion object for the case class [[org.apache.mahout.drivers.IndexedDatasetTextDelimitedWriteable]] primarily used to get a secondary constructor for - * making one [[org.apache.mahout.drivers.IndexedDatasetTextDelimitedWriteable]] from another. Used when you have a factory like [[org.apache.mahout.drivers.IndexedDatasetStore]] - * {{{ - * val id = IndexedDatasetTextDelimitedWriteable(indexedDatasetStore.read) - * }}} - */ - -object IndexedDatasetTextDelimitedWriteable { - /** Secondary constructor for [[org.apache.mahout.drivers.IndexedDataset]] */ - def apply(id2: IndexedDatasetTextDelimitedWriteable) = new IndexedDatasetTextDelimitedWriteable(id2.matrix, id2.rowIDs, id2.columnIDs, id2.writeSchema)(id2.mc) -} diff --git a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala new file mode 100644 index 0000000000..b52b5cc0ce --- /dev/null +++ b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala @@ -0,0 +1,217 @@ +/* + * 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.mahout.drivers + +import scala.collection.JavaConversions._ +import org.apache.spark.SparkContext._ +import org.apache.mahout.math.RandomAccessSparseVector +import org.apache.spark.SparkContext +import com.google.common.collect.{BiMap, HashBiMap} +import scala.collection.JavaConversions._ +import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} +import org.apache.mahout.sparkbindings._ + + +/** Extends Reader trait to supply the [[org.apache.mahout.drivers.IndexedDataset]] as the type read and a reader function for reading text delimited files as described in the [[org.apache.mahout.drivers.Schema]] + */ +trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ + /** Read in text delimited tuples from all URIs in this comma delimited source String. + * + * @param mc context for the Spark job + * @param readSchema describes the delimiters and positions of values in the text delimited file. + * @param source comma delimited URIs of text files to be read into the [[org.apache.mahout.drivers.IndexedDataset]] + * @return + */ + protected def reader(mc: SparkContext, readSchema: Schema, source: String): IndexedDataset = { + try { + val delimiter = readSchema("delim").asInstanceOf[String] + val rowIDPosition = readSchema("rowIDPosition").asInstanceOf[Int] + val columnIDPosition = readSchema("columnIDPosition").asInstanceOf[Int] + val filterPosition = readSchema("filterPosition").asInstanceOf[Int] + val filterBy = readSchema("filter").asInstanceOf[String] + // instance vars must be put into locally scoped vals when used in closures that are executed but Spark + + assert(!source.isEmpty, { + println(this.getClass.toString + ": has no files to read") + throw new IllegalArgumentException + }) + + var columns = mc.textFile(source).map({ line => line.split(delimiter)}) + + // get the rows that have a column matching the filter + columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) + + // get row and column IDs + val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) + + interactions.cache() + + // create separate collections of rowID and columnID tokens + val rowIDs = interactions.map { case (rowID, _) => rowID}.distinct().collect() + val columnIDs = interactions.map { case (_, columnID) => columnID}.distinct().collect() + + val numRows = rowIDs.size + val numColumns = columnIDs.size + + // create BiMaps for bi-directional lookup of ID by either Mahout ID or external ID + // broadcast them for access in distributed processes, so they are not recalculated in every task. + val rowIDDictionary = asOrderedDictionary(rowIDs) + val rowIDDictionary_bcast = mc.broadcast(rowIDDictionary) + + val columnIDDictionary = asOrderedDictionary(columnIDs) + val columnIDDictionary_bcast = mc.broadcast(columnIDDictionary) + + val indexedInteractions = + interactions.map { case (rowID, columnID) => + val rowIndex = rowIDDictionary_bcast.value.get(rowID).get + val columnIndex = columnIDDictionary_bcast.value.get(columnID).get + + rowIndex -> columnIndex + } + // group by IDs to form row vectors + .groupByKey().map { case (rowIndex, columnIndexes) => + val row = new RandomAccessSparseVector(numColumns) + for (columnIndex <- columnIndexes) { + row.setQuick(columnIndex, 1.0) + } + rowIndex -> row + } + .asInstanceOf[DrmRdd[Int]] + + // wrap the DrmRdd and a CheckpointedDrm, which can be used anywhere a DrmLike[Int] is needed + val drmInteractions = drmWrap[Int](indexedInteractions, numRows, numColumns) + + IndexedDataset(drmInteractions, rowIDDictionary, columnIDDictionary) + + } catch { + case cce: ClassCastException => { + println(this.getClass.toString + ": Schema has illegal values"); throw cce + } + } + } + + // this creates a BiMap from an ID collection. The ID points to an ordinal int + // which is used internal to Mahout as the row or column ID + // todo: this is a non-distributed process and the BiMap is a non-rdd based object--might be a scaling problem + private def asOrderedDictionary(entries: Array[String]): BiMap[String, Int] = { + var dictionary: BiMap[String, Int] = HashBiMap.create() + var index = 0 + for (entry <- entries) { + dictionary.forcePut(entry, index) + index += 1 + } + dictionary + } +} + +trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ + /** Read in text delimited tuples from all URIs in this comma delimited source String. + * + * @param mc context for the Spark job + * @param writeSchema describes the delimiters and positions of values in the output text delimited file. + * @param dest directory to write text delimited version of [[org.apache.mahout.drivers.IndexedDataset]] + */ + protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { + try { + val rowKeyDelim = writeSchema("rowKeyDelim").asInstanceOf[String] + val columnIdStrengthDelim = writeSchema("columnIdStrengthDelim").asInstanceOf[String] + val tupleDelim = writeSchema("tupleDelim").asInstanceOf[String] + //instance vars must be put into locally scoped vals when put into closures that are + //executed but Spark + assert (indexedDataset != null, {println(this.getClass.toString+": has no indexedDataset to write"); throw new IllegalArgumentException }) + assert (!dest.isEmpty, {println(this.getClass.toString+": has no destination or indextedDataset to write"); throw new IllegalArgumentException}) + val matrix = indexedDataset.matrix + val rowIDDictionary = indexedDataset.rowIDs + val columnIDDictionary = indexedDataset.columnIDs + + matrix.rdd.map { case (rowID, itemVector) => + + // each line is created of non-zero values with schema specified delimiters and original row and column ID tokens + // first get the external rowID token + var line: String = rowIDDictionary.inverse.get(rowID) + rowKeyDelim + + // for the rest of the row, construct the vector contents of tuples (external column ID, strength value) + for (item <- itemVector.nonZeroes()) { + line += columnIDDictionary.inverse.get(item.index) + columnIdStrengthDelim + item.get + tupleDelim + } + // drop the last delimiter, not needed to end the line + line.dropRight(1) + } + .saveAsTextFile(dest) + + }catch{ + case cce: ClassCastException => {println(this.getClass.toString+": Schema has illegal values"); throw cce} + } + } +} + +/** A combined trait that reads and writes */ +trait TDIndexedDatasetReaderWriter extends TDIndexedDatasetReader with TDIndexedDatasetWriter + +/** Reads text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. + * @param readSchema describes the delimiters and position of values in the text delimited file to be read. + * @param mc Spark context for reading files + * @note The source is supplied by Reader#readFrom . + * */ +class TextDelimitedIndexedDatasetReader(val readSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReader + +/** Writes text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. + * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. + * @param mc Spark context for reading files + * @note the destination is supplied by Writer#writeTo trait method + * */ +class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetWriter + +/** Reads and writes text delimited files to/from an IndexedDataset. Classes are needed to supply trait params in their constructor. + * @param readSchema describes the delimiters and position of values in the text delimited file(s) to be read. + * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. + * @param mc Spark context for reading the files, may be implicitly defined. + * */ +class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReaderWriter + +/** A version of IndexedDataset that has it's own writeTo method from a Writer trait. This is an alternative to creating + * a Writer based stand-alone class for writing. Consider it experimental allowing similar semantics to drm.writeDrm(). + * Experimental because it's not clear that it is simpler or more intuitive and since IndexedDatasetTextDelimitedWriteables + * are probably short lived in terms of lines of code so complexity may be moot. + * @param matrix the data + * @param rowIDs bi-directional dictionary for rows of external IDs to internal ordinal Mahout IDs. + * @param columnIDs bi-directional dictionary for columns of external IDs to internal ordinal Mahout IDs. + * @param writeSchema contains params for the schema/format or the written text delimited file. + * @param mc mahout distributed context (SparkContext) may be implicitly defined. + * */ +class IndexedDatasetTextDelimitedWriteable(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int], + val writeSchema: Schema)(implicit val mc: SparkContext) + extends IndexedDataset(matrix, rowIDs, columnIDs) with TDIndexedDatasetWriter { + + def writeTo(dest: String): Unit = { + writeTo(this, dest) + } +} + +/** + * Companion object for the case class [[org.apache.mahout.drivers.IndexedDatasetTextDelimitedWriteable]] primarily used to get a secondary constructor for + * making one [[org.apache.mahout.drivers.IndexedDatasetTextDelimitedWriteable]] from another. Used when you have a factory like [[org.apache.mahout.drivers.TextDelimitedIndexedDatasetReader]] + * {{{ + * val id = IndexedDatasetTextDelimitedWriteable(indexedDatasetReader.readFrom(source)) + * }}} + */ + +object IndexedDatasetTextDelimitedWriteable { + /** Secondary constructor for [[org.apache.mahout.drivers.IndexedDataset]] */ + def apply(id2: IndexedDatasetTextDelimitedWriteable) = new IndexedDatasetTextDelimitedWriteable(id2.matrix, id2.rowIDs, id2.columnIDs, id2.writeSchema)(id2.mc) +} diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala deleted file mode 100644 index b98d48f6fd..0000000000 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriver$Test.scala +++ /dev/null @@ -1,140 +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.mahout.drivers - -import org.scalatest.FunSuite - -class ItemSimilarityDriver$Test extends FunSuite { - - test ("Running some text delimited files through the import/cooccurrence/export"){ - - val exampleCsvlogStatements = Array( - "12569537329,user1,item1,\"view\"", - "12569537329,user1,item2,\"view\"", - "12569537329,user1,item2,\"like\"", - "12569537329,user1,item3,\"like\"", - "12569537329,user2,item2,\"view\"", - "12569537329,user2,item2,\"like\"", - "12569537329,user3,item1,\"like\"", - "12569537329,user3,item3,\"view\"", - "12569537329,user3,item3,\"like\"") - - val exampleTsvLogStatements = Array( - "12569537329\tuser1\titem1\t\"view\"", - "12569537329\tuser1\titem2\t\"view\"", - "12569537329\tuser1\titem2\t\"like\"", - "12569537329\tuser1\titem3\t\"like\"", - "12569537329\tuser2\titem2\t\"view\"", - "12569537329\tuser2\titem2\t\"like\"", - "12569537329\tuser3\titem1\t\"like\"", - "12569537329\tuser3\titem3\t\"view\"", - "12569537329\tuser3\titem3\t\"like\"") - - - val csvLogStatements = Array( - "u1,purchase,iphone", - "u1,purchase,ipad", - "u2,purchase,nexus", - "u2,purchase,galaxy", - "u3,purchase,surface", - "u4,purchase,iphone", - "u4,purchase,galaxy", - "u1,view,iphone", - "u1,view,ipad", - "u1,view,nexus", - "u1,view,galaxy", - "u2,view,iphone", - "u2,view,ipad", - "u2,view,nexus", - "u2,view,galaxy", - "u3,view,surface", - "u3,view,nexus", - "u4,view,iphone", - "u4,view,ipad", - "u4,view,galaxy") - - val tsvLogStatements = Array( - "u1\tpurchase\tiphone", - "u1\tpurchase\tipad", - "u2\tpurchase\tnexus", - "u2\tpurchase\tgalaxy", - "u3\tpurchase\tsurface", - "u4\tpurchase\tiphone", - "u4\tpurchase\tgalaxy", - "u1\tview\tiphone", - "u1\tview\tipad", - "u1\tview\tnexus", - "u1\tview\tgalaxy", - "u2\tview\tiphone", - "u2\tview\tipad", - "u2\tview\tnexus", - "u2\tview\tgalaxy", - "u3\tview\tsurface", - "u3\tview\tnexus", - "u4\tview\tiphone", - "u4\tview\tipad", - "u4\tview\tgalaxy") - - /* - //Clustered Spark and HDFS - ItemSimilarityDriver.main(Array( - "--input", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/cf-data.txt", - "--output", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/indicatorMatrices/", - "--master", "spark://occam4:7077", - "--filter1", "purchase", - "--filter2", "view", - "--inDelim", ",", - "--itemIDPosition", "2", - "--rowIDPosition", "0", - "--filterPosition", "1" - )) - */ - // local multi-threaded Spark with HDFS using large dataset - // todo: not sure how to handle build testing on HDFS maybe make into an integration test - // or example. - /* ItemSimilarityDriver.main(Array( - "--input", "hdfs://occam4:54310/user/pat/xrsj/ratings_data.txt", - "--output", "hdfs://occam4:54310/user/pat/xrsj/indicatorMatrices/", - "--master", "local[4]", - "--filter1", "purchase", - "--filter2", "view", - "--inDelim", ",", - "--itemIDPosition", "2", - "--rowIDPosition", "0", - "--filterPosition", "1" - )) - */ - - // local multi-threaded Spark with local FS, suitable for build tests but need better location for data - // todo: remove absolute path - // todo: check computed value or it's not much of a test. - ItemSimilarityDriver.main(Array( - "--input", "/Users/pat/big-data/tmp/cf-data.txt", - "--output", "/Users/pat/big-data/tmp/indicatorMatrices/", - "--master", "local[4]", - "--filter1", "purchase", - "--filter2", "view", - "--inDelim", ",", - "--itemIDPosition", "2", - "--rowIDPosition", "0", - "--filterPosition", "1" - )) - - } - -} diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala new file mode 100644 index 0000000000..ab74ccbd2c --- /dev/null +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -0,0 +1,174 @@ +/* + * 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.mahout.drivers + +import org.scalatest.FunSuite +import org.apache.mahout.sparkbindings._ +import org.apache.mahout.sparkbindings.test.MahoutLocalContext +import org.apache.mahout.test.MahoutSuite + +class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLocalContext { + + final val SelfSimilairtyTSV = Set( + "galaxy\tnexus:1.7260924347106847", + "ipad\tiphone:1.7260924347106847", + "nexus\tgalaxy:1.7260924347106847", + "iphone\tipad:1.7260924347106847", + "surface") + + final val CrossSimilarityTSV = Set("" + + "nexus\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,surface:0.6795961471815897,galaxy:1.7260924347106847", + "ipad\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", + "surface\tsurface:4.498681156950466", + "iphone\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", + "galaxy\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847") + + /* + //Clustered Spark and HDFS + ItemSimilarityDriver.main(Array( + "--input", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/cf-data.txt", + "--output", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/indicatorMatrices/", + "--master", "spark://occam4:7077", + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1" + )) +*/ + // local multi-threaded Spark with HDFS using large dataset + // todo: not sure how to handle build testing on HDFS maybe make into an integration test + // or example. + /* ItemSimilarityDriver.main(Array( + "--input", "hdfs://occam4:54310/user/pat/xrsj/ratings_data.txt", + "--output", "hdfs://occam4:54310/user/pat/xrsj/indicatorMatrices/", + "--master", "local[4]", + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1" + )) + */ + + test ("running simple, non-full-spec CSV through"){ + + val InFile = "tmp/in-file.csv" + val OutPath = "tmp/indicator-matrices" + + val lines = Array( + "u1,purchase,iphone", + "u1,purchase,ipad", + "u2,purchase,nexus", + "u2,purchase,galaxy", + "u3,purchase,surface", + "u4,purchase,iphone", + "u4,purchase,galaxy", + "u1,view,iphone", + "u1,view,ipad", + "u1,view,nexus", + "u1,view,galaxy", + "u2,view,iphone", + "u2,view,ipad", + "u2,view,nexus", + "u2,view,galaxy", + "u3,view,surface", + "u3,view,nexus", + "u4,view,iphone", + "u4,view,ipad", + "u4,view,galaxy") + + val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) + + afterEach()// clean up before running the driver, it should handle the Spark conf and context + + // local multi-threaded Spark with default FS, suitable for build tests but need better location for data + ItemSimilarityDriver.main(Array( + "--input", InFile, + "--output", OutPath, + "--master", masterUrl, + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1")) + + beforeEach()// restart the test context to read the output of the driver + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == SelfSimilairtyTSV) + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] + assert (crossIndicatorLines == CrossSimilarityTSV) + } + + + + test ("Running TSV files through"){ + + val InFile = "tmp/in-file.tsv" + val OutPath = "tmp/indicator-matrices" + + val lines = Array( + "u1\tpurchase\tiphone", + "u1\tpurchase\tipad", + "u2\tpurchase\tnexus", + "u2\tpurchase\tgalaxy", + "u3\tpurchase\tsurface", + "u4\tpurchase\tiphone", + "u4\tpurchase\tgalaxy", + "u1\tview\tiphone", + "u1\tview\tipad", + "u1\tview\tnexus", + "u1\tview\tgalaxy", + "u2\tview\tiphone", + "u2\tview\tipad", + "u2\tview\tnexus", + "u2\tview\tgalaxy", + "u3\tview\tsurface", + "u3\tview\tnexus", + "u4\tview\tiphone", + "u4\tview\tipad", + "u4\tview\tgalaxy") + + val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) + + afterEach()// clean up before running the driver, it should handle the Spark conf and context + + // local multi-threaded Spark with default FS, suitable for build tests but need better location for data + ItemSimilarityDriver.main(Array( + "--input", InFile, + "--output", OutPath, + "--master", masterUrl, + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", "[,\t]", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1")) + + beforeEach()// restart the test context to read the output of the driver + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == SelfSimilairtyTSV) + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] + assert (crossIndicatorLines == CrossSimilarityTSV) + + } + +} diff --git a/spark/src/test/scala/org/apache/mahout/sparkbindings/test/MahoutLocalContext.scala b/spark/src/test/scala/org/apache/mahout/sparkbindings/test/MahoutLocalContext.scala index d9e89bceda..ad7f4c31ad 100644 --- a/spark/src/test/scala/org/apache/mahout/sparkbindings/test/MahoutLocalContext.scala +++ b/spark/src/test/scala/org/apache/mahout/sparkbindings/test/MahoutLocalContext.scala @@ -10,11 +10,13 @@ trait MahoutLocalContext extends MahoutSuite with LoggerConfiguration { this: Suite => protected implicit var mahoutCtx: DistributedContext = _ + protected var masterUrl = null.asInstanceOf[String] override protected def beforeEach() { super.beforeEach() - mahoutCtx = mahoutSparkContext(masterUrl = "local[2]", + masterUrl = "local[2]" + mahoutCtx = mahoutSparkContext(masterUrl = this.masterUrl, appName = "MahoutLocalContext", // Do not run MAHOUT_HOME jars in unit tests. addMahoutJars = false, From ab8009f6176f0c21a07e15cc5cc8a9717dd7cc4c Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 25 Jun 2014 08:41:54 -0700 Subject: [PATCH 12/38] adding more tests for ItemSimilarityDriver --- .../apache/mahout/drivers/FileSysUtils.scala | 51 +++- .../mahout/drivers/ItemSimilarityDriver.scala | 15 +- .../drivers/TextDelimitedReaderWriter.scala | 7 +- .../drivers/ItemSimilarityDriverSuite.scala | 231 ++++++++++++++++-- 4 files changed, 266 insertions(+), 38 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala b/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala index e8491f2ea9..85a769f71c 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala @@ -17,18 +17,55 @@ package org.apache.mahout.drivers +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{Path, FileStatus, FileSystem} + /** * Returns a [[java.lang.String]]comma delimited list of URIs discovered based on parameters in the constructor. * The String is formatted to be input into [[org.apache.spark.SparkContext.textFile()]] * - * @param pathURI Where to start looking for inFiles, only HDFS and local URI are currently - * supported - * @param filePattern regex that must match the entire filename to have the file included in the returned list + * @param pathURI Where to start looking for inFiles, only HDFS is currently + * supported. The pathURI may be a list of comma delimited URIs like those supported + * by Spark + * @param filePattern regex that must match the entire filename to have the file returned * @param recursive true traverses the filesystem recursively */ -case class FileSysUtils(pathURI: String, filePattern: String = "", recursive: Boolean = false) { - // todo: There is an HDFS filestatus method that collects multiple inFiles, see if this is the right thing to use - // todo: check to see if the input is a supported URI for collection or recursive search but just pass through otherwise - def uris = {pathURI} +case class FileSysUtils(pathURI: String, filePattern: String = ".*", recursive: Boolean = false) { + + val conf = new Configuration() + val fs = FileSystem.get(conf) + + /** returns a string of comma delimited URIs matching the filePattern */ + def uris :String = { + if(recursive){ + val pathURIs = pathURI.split(",") + var files = "" + for ( uri <- pathURIs ){ + files = findFiles(uri, filePattern, files) + } + if (files.length > 0 && files.endsWith(",")) files.dropRight(1) // drop the last comma + files + }else{ + pathURI + } + } + + /** find matching files in the dir, recursively call self when another directory is found */ + def findFiles(dir: String, filePattern :String = ".*", files : String = ""): String = { + val fileStatuses: Array[FileStatus] = fs.listStatus (new Path(dir)) + var f :String = files + for (fileStatus <- fileStatuses ){ + if (fileStatus.getPath().getName().matches(filePattern) + && !fileStatus.isDir){// found a file + if (fileStatus.getLen() != 0) { + // file is not empty + f = f + fileStatus.getPath.toUri.toString + "," + } + }else if (fileStatus.isDir){ + f = findFiles(fileStatus.getPath.toString, filePattern, f) + } + } + f + } } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index faf69496d5..a7d31c7907 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -160,8 +160,10 @@ object ItemSimilarityDriver extends MahoutDriver { help("help") abbr ("h") text ("prints this usage text\n") checkConfig { c => - if (c.filterPosition == c.itemIDPosition || c.filterPosition == c.rowIDPosition || c.rowIDPosition == c.itemIDPosition) failure("The row, item, and filter positions must be" + - " unique.") else success + if (c.filterPosition == c.itemIDPosition + || c.filterPosition == c.rowIDPosition + || c.rowIDPosition == c.itemIDPosition) + failure("The row, item, and filter positions must be unique.") else success } //check for option consistency, probably driver specific @@ -243,7 +245,14 @@ object ItemSimilarityDriver extends MahoutDriver { val indexedDatasets = readIndexedDatasets - val indicatorMatrices = CooccurrenceAnalysis.cooccurrences(indexedDatasets(0).matrix, options.randomSeed, options.maxSimilaritiesPerItem, options.maxPrefs, Array(indexedDatasets(1).matrix)) + // todo: allow more than one cross-similarity matrix? + val indicatorMatrices = { + if (indexedDatasets.length > 1) { + CooccurrenceAnalysis.cooccurrences(indexedDatasets(0).matrix, options.randomSeed, options.maxSimilaritiesPerItem, options.maxPrefs, Array(indexedDatasets(1).matrix)) + } else { + CooccurrenceAnalysis.cooccurrences(indexedDatasets(0).matrix, options.randomSeed, options.maxSimilaritiesPerItem, options.maxPrefs) + } + } // self similarity // the next two lines write the drm using a Writer class diff --git a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala index b52b5cc0ce..ecaba6bf24 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala @@ -53,8 +53,11 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ var columns = mc.textFile(source).map({ line => line.split(delimiter)}) - // get the rows that have a column matching the filter - columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) + // -1 means no filter in the input text, take them all + if(filterPosition != -1) { + // get the rows that have a column matching the filter + columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) + } // get row and column IDs val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index ab74ccbd2c..1e636c9bc0 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -17,6 +17,8 @@ package org.apache.mahout.drivers +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{Path, FileSystem} import org.scalatest.FunSuite import org.apache.mahout.sparkbindings._ import org.apache.mahout.sparkbindings.test.MahoutLocalContext @@ -97,9 +99,9 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) - afterEach()// clean up before running the driver, it should handle the Spark conf and context + afterEach // clean up before running the driver, it should handle the Spark conf and context - // local multi-threaded Spark with default FS, suitable for build tests but need better location for data + // local multi-threaded Spark with default HDFS ItemSimilarityDriver.main(Array( "--input", InFile, "--output", OutPath, @@ -111,7 +113,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--rowIDPosition", "0", "--filterPosition", "1")) - beforeEach()// restart the test context to read the output of the driver + beforeEach // restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] assert(indicatorLines == SelfSimilairtyTSV) val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] @@ -126,32 +128,32 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc val OutPath = "tmp/indicator-matrices" val lines = Array( - "u1\tpurchase\tiphone", - "u1\tpurchase\tipad", - "u2\tpurchase\tnexus", - "u2\tpurchase\tgalaxy", - "u3\tpurchase\tsurface", - "u4\tpurchase\tiphone", - "u4\tpurchase\tgalaxy", - "u1\tview\tiphone", - "u1\tview\tipad", - "u1\tview\tnexus", - "u1\tview\tgalaxy", - "u2\tview\tiphone", - "u2\tview\tipad", - "u2\tview\tnexus", - "u2\tview\tgalaxy", - "u3\tview\tsurface", - "u3\tview\tnexus", - "u4\tview\tiphone", - "u4\tview\tipad", - "u4\tview\tgalaxy") + "u1\tpurchase\tiphone", + "u1\tpurchase\tipad", + "u2\tpurchase\tnexus", + "u2\tpurchase\tgalaxy", + "u3\tpurchase\tsurface", + "u4\tpurchase\tiphone", + "u4\tpurchase\tgalaxy", + "u1\tview\tiphone", + "u1\tview\tipad", + "u1\tview\tnexus", + "u1\tview\tgalaxy", + "u2\tview\tiphone", + "u2\tview\tipad", + "u2\tview\tnexus", + "u2\tview\tgalaxy", + "u3\tview\tsurface", + "u3\tview\tnexus", + "u4\tview\tiphone", + "u4\tview\tipad", + "u4\tview\tgalaxy") val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) - afterEach()// clean up before running the driver, it should handle the Spark conf and context + afterEach // clean up before running the driver, it should handle the Spark conf and context - // local multi-threaded Spark with default FS, suitable for build tests but need better location for data + // local multi-threaded Spark with default HDFS ItemSimilarityDriver.main(Array( "--input", InFile, "--output", OutPath, @@ -163,7 +165,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--rowIDPosition", "0", "--filterPosition", "1")) - beforeEach()// restart the test context to read the output of the driver + beforeEach // restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] assert(indicatorLines == SelfSimilairtyTSV) val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] @@ -171,4 +173,181 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } + test ("Running log files through"){ + + val InFile = "tmp/in-file.log" + val OutPath = "tmp/indicator-matrices" + + val lines = Array( + "2014-06-23 14:46:53.115\tu1\tpurchase\trandom text\tiphone", + "2014-06-23 14:46:53.115\tu1\tpurchase\trandom text\tipad", + "2014-06-23 14:46:53.115\tu2\tpurchase\trandom text\tnexus", + "2014-06-23 14:46:53.115\tu2\tpurchase\trandom text\tgalaxy", + "2014-06-23 14:46:53.115\tu3\tpurchase\trandom text\tsurface", + "2014-06-23 14:46:53.115\tu4\tpurchase\trandom text\tiphone", + "2014-06-23 14:46:53.115\tu4\tpurchase\trandom text\tgalaxy", + "2014-06-23 14:46:53.115\tu1\tview\trandom text\tiphone", + "2014-06-23 14:46:53.115\tu1\tview\trandom text\tipad", + "2014-06-23 14:46:53.115\tu1\tview\trandom text\tnexus", + "2014-06-23 14:46:53.115\tu1\tview\trandom text\tgalaxy", + "2014-06-23 14:46:53.115\tu2\tview\trandom text\tiphone", + "2014-06-23 14:46:53.115\tu2\tview\trandom text\tipad", + "2014-06-23 14:46:53.115\tu2\tview\trandom text\tnexus", + "2014-06-23 14:46:53.115\tu2\tview\trandom text\tgalaxy", + "2014-06-23 14:46:53.115\tu3\tview\trandom text\tsurface", + "2014-06-23 14:46:53.115\tu3\tview\trandom text\tnexus", + "2014-06-23 14:46:53.115\tu4\tview\trandom text\tiphone", + "2014-06-23 14:46:53.115\tu4\tview\trandom text\tipad", + "2014-06-23 14:46:53.115\tu4\tview\trandom text\tgalaxy") + + val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) + + afterEach // clean up before running the driver, it should handle the Spark conf and context + + // local multi-threaded Spark with default HDFS + ItemSimilarityDriver.main(Array( + "--input", InFile, + "--output", OutPath, + "--master", masterUrl, + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", "\t", + "--itemIDPosition", "4", + "--rowIDPosition", "1", + "--filterPosition", "2")) + + beforeEach // restart the test context to read the output of the driver + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == SelfSimilairtyTSV) + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] + assert (crossIndicatorLines == CrossSimilarityTSV) + + } + + test ("Running legacy files through"){ + + val InDir = "tmp/in-dir/" + val InFilename = "in-file.tsv" + val InPath = InDir + InFilename + + val OutPath = "tmp/indicator-matrices" + + val lines = Array( + "0,0,1", + "0,1,1", + "1,2,1", + "1,3,1", + "2,4,1", + "3,0,1", + "3,3,1") + + val Answer = Set( + "0\t1:1.7260924347106847", + "3\t2:1.7260924347106847", + "1\t0:1.7260924347106847", + "4", + "2\t3:1.7260924347106847") + + // this creates one part-0000 file in the directory + mahoutCtx.parallelize(lines).coalesce(1, shuffle=true).saveAsTextFile(InDir) + + // to change from using part files to a single .tsv file we'll need to use HDFS + val conf = new Configuration(); + val fs = FileSystem.get(conf); + //rename part-00000 to something.tsv + fs.rename(new Path(InDir + "part-00000"), new Path(InPath)) + + afterEach // clean up before running the driver, it should handle the Spark conf and context + + // local multi-threaded Spark with default HDFS + ItemSimilarityDriver.main(Array( + "--input", InPath, + "--output", OutPath, + "--master", masterUrl)) + + beforeEach // restart the test context to read the output of the driver + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == Answer) + + } + + test("recursive file discovery using filename patterns"){ + //directory structure using the following + // tmp/data/m1.tsv + // tmp/data/more-data/another-dir/m2.tsv + val M1Lines = Array( + "u1\tpurchase\tiphone", + "u1\tpurchase\tipad", + "u2\tpurchase\tnexus", + "u2\tpurchase\tgalaxy", + "u3\tpurchase\tsurface", + "u4\tpurchase\tiphone", + "u4\tpurchase\tgalaxy", + "u1\tview\tiphone") + + val M2Lines = Array( + "u1\tview\tipad", + "u1\tview\tnexus", + "u1\tview\tgalaxy", + "u2\tview\tiphone", + "u2\tview\tipad", + "u2\tview\tnexus", + "u2\tview\tgalaxy", + "u3\tview\tsurface", + "u3\tview\tnexus", + "u4\tview\tiphone", + "u4\tview\tipad", + "u4\tview\tgalaxy") + + val InFilenameM1 = "m1.tsv" + val InDirM1 = "tmp/data/" + val InPathM1 = InDirM1 + InFilenameM1 + val InFilenameM2 = "m2.tsv" + val InDirM2 = "tmp/data/more-data/another-dir/" + val InPathM2 = InDirM2 + InFilenameM2 + + val InPathStart = "tmp/data/" + val OutPath = "tmp/indicator-matrices" + + // this creates one part-0000 file in the directory + mahoutCtx.parallelize(M1Lines).coalesce(1, shuffle=true).saveAsTextFile(InDirM1) + + // to change from using part files to a single .tsv file we'll need to use HDFS + val conf = new Configuration(); + val fs = FileSystem.get(conf); + //rename part-00000 to something.tsv + fs.rename(new Path(InDirM1 + "part-00000"), new Path(InPathM1)) + + // this creates one part-0000 file in the directory + mahoutCtx.parallelize(M2Lines).coalesce(1, shuffle=true).saveAsTextFile(InDirM2) + + // to change from using part files to a single .tsv file we'll need to use HDFS + //rename part-00000 to tmp/some-location/something.tsv + fs.rename(new Path(InDirM2 + "part-00000"), new Path(InPathM2)) + + // local multi-threaded Spark with default FS, suitable for build tests but need better location for data + + afterEach // clean up before running the driver, it should handle the Spark conf and context + + ItemSimilarityDriver.main(Array( + "--input", InPathStart, + "--output", OutPath, + "--master", masterUrl, + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", "[,\t]", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1", + "--filenamePattern", "m..tsv", + "--recursive")) + + beforeEach()// restart the test context to read the output of the driver + val indicatorLines = mahoutCtx.textFile(OutPath + "/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == SelfSimilairtyTSV) + val crossIndicatorLines = mahoutCtx.textFile(OutPath + "/cross-indicator-matrix/").collect.toSet[String] + assert (crossIndicatorLines == CrossSimilarityTSV) + + } + } From 9a02e2a5ea8540723c1bfc6ea01b045bb4175922 Mon Sep 17 00:00:00 2001 From: pferrel Date: Wed, 25 Jun 2014 09:57:55 -0700 Subject: [PATCH 13/38] remove tmp after all tests, fixed dangling comma in input file list --- .../apache/mahout/drivers/FileSysUtils.scala | 2 +- .../mahout/drivers/IndexedDatasetTest.scala | 25 --------- .../drivers/ItemSimilarityDriverSuite.scala | 52 ++++++++++++------- 3 files changed, 33 insertions(+), 46 deletions(-) delete mode 100644 spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala diff --git a/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala b/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala index 85a769f71c..654f116f0d 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/FileSysUtils.scala @@ -44,7 +44,7 @@ case class FileSysUtils(pathURI: String, filePattern: String = ".*", recursive: for ( uri <- pathURIs ){ files = findFiles(uri, filePattern, files) } - if (files.length > 0 && files.endsWith(",")) files.dropRight(1) // drop the last comma + if (files.length > 0 && files.endsWith(",")) files = files.dropRight(1) // drop the last comma files }else{ pathURI diff --git a/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala b/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala deleted file mode 100644 index 2b2a960567..0000000000 --- a/spark/src/test/scala/org/apache/mahout/drivers/IndexedDatasetTest.scala +++ /dev/null @@ -1,25 +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.mahout.drivers - -import org.scalatest.FunSuite - -class IndexedDatasetTest extends FunSuite { - // todo: put some tests here! - -} diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index 1e636c9bc0..3e7f4c35d5 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -40,8 +40,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "iphone\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", "galaxy\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847") + final val TmpDir = "tmp/" // all IO going to whatever the default HDFS config is pointing to + /* - //Clustered Spark and HDFS + //Clustered Spark and HDFS, not a good everyday build test ItemSimilarityDriver.main(Array( "--input", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/cf-data.txt", "--output", "hdfs://occam4:54310/user/pat/spark-itemsimilarity/indicatorMatrices/", @@ -55,8 +57,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc )) */ // local multi-threaded Spark with HDFS using large dataset - // todo: not sure how to handle build testing on HDFS maybe make into an integration test - // or example. + // not a good build test. /* ItemSimilarityDriver.main(Array( "--input", "hdfs://occam4:54310/user/pat/xrsj/ratings_data.txt", "--output", "hdfs://occam4:54310/user/pat/xrsj/indicatorMatrices/", @@ -72,8 +73,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc test ("running simple, non-full-spec CSV through"){ - val InFile = "tmp/in-file.csv" - val OutPath = "tmp/indicator-matrices" + val InFile = TmpDir + "in-file.csv" + val OutPath = TmpDir + "indicator-matrices" val lines = Array( "u1,purchase,iphone", @@ -97,6 +98,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "u4,view,ipad", "u4,view,galaxy") + // this will create multiple part-xxxxx files in the InFile dir but other tests will + // take account of one actual file val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) afterEach // clean up before running the driver, it should handle the Spark conf and context @@ -124,8 +127,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc test ("Running TSV files through"){ - val InFile = "tmp/in-file.tsv" - val OutPath = "tmp/indicator-matrices" + val InFile = TmpDir + "in-file.tsv" + val OutPath = TmpDir + "indicator-matrices" val lines = Array( "u1\tpurchase\tiphone", @@ -149,6 +152,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "u4\tview\tipad", "u4\tview\tgalaxy") + // this will create multiple part-xxxxx files in the InFile dir but other tests will + // take account of one actual file val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) afterEach // clean up before running the driver, it should handle the Spark conf and context @@ -175,8 +180,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc test ("Running log files through"){ - val InFile = "tmp/in-file.log" - val OutPath = "tmp/indicator-matrices" + val InFile = TmpDir + "in-file.log" + val OutPath = TmpDir + "indicator-matrices" val lines = Array( "2014-06-23 14:46:53.115\tu1\tpurchase\trandom text\tiphone", @@ -200,6 +205,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "2014-06-23 14:46:53.115\tu4\tview\trandom text\tipad", "2014-06-23 14:46:53.115\tu4\tview\trandom text\tgalaxy") + // this will create multiple part-xxxxx files in the InFile dir but other tests will + // take account of one actual file val linesRdd = mahoutCtx.parallelize(lines).saveAsTextFile(InFile) afterEach // clean up before running the driver, it should handle the Spark conf and context @@ -226,11 +233,11 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc test ("Running legacy files through"){ - val InDir = "tmp/in-dir/" + val InDir = TmpDir + "in-dir/" val InFilename = "in-file.tsv" val InPath = InDir + InFilename - val OutPath = "tmp/indicator-matrices" + val OutPath = TmpDir + "indicator-matrices" val lines = Array( "0,0,1", @@ -252,8 +259,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc mahoutCtx.parallelize(lines).coalesce(1, shuffle=true).saveAsTextFile(InDir) // to change from using part files to a single .tsv file we'll need to use HDFS - val conf = new Configuration(); - val fs = FileSystem.get(conf); + val fs = FileSystem.get(new Configuration()) //rename part-00000 to something.tsv fs.rename(new Path(InDir + "part-00000"), new Path(InPath)) @@ -300,21 +306,20 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "u4\tview\tgalaxy") val InFilenameM1 = "m1.tsv" - val InDirM1 = "tmp/data/" + val InDirM1 = TmpDir + "data/" val InPathM1 = InDirM1 + InFilenameM1 val InFilenameM2 = "m2.tsv" - val InDirM2 = "tmp/data/more-data/another-dir/" + val InDirM2 = TmpDir + "data/more-data/another-dir/" val InPathM2 = InDirM2 + InFilenameM2 - val InPathStart = "tmp/data/" - val OutPath = "tmp/indicator-matrices" + val InPathStart = TmpDir + "data/" + val OutPath = TmpDir + "indicator-matrices" // this creates one part-0000 file in the directory mahoutCtx.parallelize(M1Lines).coalesce(1, shuffle=true).saveAsTextFile(InDirM1) // to change from using part files to a single .tsv file we'll need to use HDFS - val conf = new Configuration(); - val fs = FileSystem.get(conf); + val fs = FileSystem.get(new Configuration()) //rename part-00000 to something.tsv fs.rename(new Path(InDirM1 + "part-00000"), new Path(InPathM1)) @@ -335,7 +340,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--master", masterUrl, "--filter1", "purchase", "--filter2", "view", - "--inDelim", "[,\t]", + "--inDelim", "\t", "--itemIDPosition", "2", "--rowIDPosition", "0", "--filterPosition", "1", @@ -347,7 +352,14 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc assert(indicatorLines == SelfSimilairtyTSV) val crossIndicatorLines = mahoutCtx.textFile(OutPath + "/cross-indicator-matrix/").collect.toSet[String] assert (crossIndicatorLines == CrossSimilarityTSV) + } + + override def afterAll = { + // remove TmpDir + val fs = FileSystem.get(new Configuration()) + fs.delete(new Path(TmpDir), true) // delete recursively + super.afterAll } } From 3c343ff18600f0a0e59f5bfd63bd86db0db0e8c5 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 26 Jun 2014 15:19:48 -0700 Subject: [PATCH 14/38] changes to pom, mahout driver script, and cleaned up help text --- bin/mahout | 16 ++++++-- spark/pom.xml | 21 ++++++++++ .../mahout/drivers/ItemSimilarityDriver.scala | 40 ++++++++++--------- 3 files changed, 54 insertions(+), 23 deletions(-) diff --git a/bin/mahout b/bin/mahout index e0720acf0f..d312cbc2e4 100755 --- a/bin/mahout +++ b/bin/mahout @@ -84,6 +84,9 @@ if [ "$1" == "spark-shell" ]; then SPARK=1 fi +if [ "$1" == "spark-itemsimilarity" ]; then + SPARK=1 +fi if [ "$MAHOUT_CORE" != "" ]; then IS_CORE=1 @@ -156,14 +159,14 @@ then CLASSPATH=${CLASSPATH}:$f; done fi - + # add scala dev target - for f in $MAHOUT_HOME/math-scala/target/mahout-math-scala-*.jar ; do + for f in $MAHOUT_HOME/math-scala/target/mahout-math-scala-*.jar ; do CLASSPATH=${CLASSPATH}:$f; done - # add spark-shell -- if we requested shell + # add spark-shell -- if we requested shell or other spark CLI driver if [ "$SPARK" == "1" ]; then for f in $MAHOUT_HOME/mrlegacy/target/mahout-mrlegacy-*.jar ; do @@ -183,7 +186,7 @@ then SPARK_CLASSPATH=$("${SPARK_CP_BIN}" 2>/dev/null) CLASSPATH="${CLASSPATH}:${SPARK_CLASSPATH}" else - echo "Cannot find Spark classpath." + echo "Cannot find Spark classpath. Is 'SPARK_HOME' set?" exit -1 fi @@ -228,6 +231,11 @@ case "$1" in "$JAVA" $JAVA_HEAP_MAX -classpath "$CLASSPATH" "org.apache.mahout.sparkbindings.shell.Main" $@ stty sane; stty $save_stty ;; + # Spark CLI drivers go here + (spark-itemsimilarity) + shift + "$JAVA" $JAVA_HEAP_MAX -classpath "$CLASSPATH" "org.apache.mahout.drivers.ItemSimilarityDriver" "$@" + ;; (*) # default log directory & file diff --git a/spark/pom.xml b/spark/pom.xml index 8af262835a..daa4aeceb4 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -83,6 +83,27 @@ + + + + org.apache.maven.plugins + maven-assembly-plugin + + + job + package + + single + + + + src/main/assembly/job.xml + + + + + + org.apache.maven.plugins diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index a7d31c7907..fa0bb79337 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -47,43 +47,45 @@ object ItemSimilarityDriver extends MahoutDriver { * @param args Command line args, if empty a help message is printed. */ override def main(args: Array[String]): Unit = { - val parser = new MahoutOptionParser[Options]("ItemSimilarity") { - head("ItemSimilarity", "Spark") + val parser = new MahoutOptionParser[Options]("spark-itemsimilarity") { + head("spark-itemsimilarity", "Mahout 1.0-SNAPSHOT") //Input output options, non-driver specific note("Input, output options") opt[String]('i', "input") required() action { (x, options) => options.copy(input = x) - } text ("Path for input. It may be a filename or directory name and can be a local file path or an HDFS URI (required).") + } text ("Input path, may be a filename, directory name, or comma delimited list of HDFS supported URIs (required)") opt[String]('o', "output") required() action { (x, options) => if (x.endsWith("/")) // todo: check to see if HDFS allows MS-Windows backslashes locally? options.copy(output = x) else options.copy(output = x + "/") - } text ("Output will be in sub-directories stored here so this must be a directory path (required).") + } text ("Path for output, any local or HDFS supported URI (required).") //Algorithm control options--driver specific note("\nAlgorithm control options:") - opt[String]("master") abbr ("ma") text ("URL for the Spark Master. (optional). Default: 'local'") action { (x, options) => + opt[String]("master") abbr ("ma") text ("Spark Master URL (optional). Default: \"local\". Note that you can specify the number of cores to get a performance improvement, for example \"local[4]\"") action { (x, options) => options.copy(master = x) } opt[Int]("maxPrefs") abbr ("mppu") action { (x, options) => options.copy(maxPrefs = x) - } text ("Max number of preferences to consider per user or item, users or items with more preferences will be sampled down (optional). Default: 500") validate { x => + } text ("Max number of preferences to consider per user (optional). Default: 500") validate { x => if (x > 0) success else failure("Option --maxPrefs must be > 0") } +/** not implemented in CooccurrenceAnalysis.cooccurrence opt[Int]("minPrefs") abbr ("mp") action { (x, options) => options.copy(minPrefs = x) } text ("Ignore users with less preferences than this (optional). Default: 1") validate { x => if (x > 0) success else failure("Option --minPrefs must be > 0") } +*/ opt[Int]('m', "maxSimilaritiesPerItem") action { (x, options) => options.copy(maxSimilaritiesPerItem = x) - } text ("Try to cap the number of similar items for each item to this number (optional). Default: 100") validate { x => + } text ("Limit the number of similarities per item to this number (optional). Default: 100") validate { x => if (x > 0) success else failure("Option --maxSimilaritiesPerItem must be > 0") } @@ -96,17 +98,17 @@ object ItemSimilarityDriver extends MahoutDriver { //Input text file schema--not driver specific but input data specific, tuples input, // not drms note("\nInput text file schema options:") - opt[String]("inDelim") abbr ("d") text ("Input delimiter character (optional). Default: '\\t'") action { (x, options) => + opt[String]("inDelim") abbr ("id") text ("Input delimiter character (optional). Default: \"[,\\t]\"") action { (x, options) => options.copy(inDelim = x) } opt[String]("filter1") abbr ("f1") action { (x, options) => options.copy(filter1 = x) - } text ("String whose presence indicates a datum for the primary item set, can be a regex (optional). Default: no filtered is applied, all is used") + } text ("String (or regex) whose presence indicates a datum for the primary item set (optional). Default: no filter, all data is used") opt[String]("filter2") abbr ("f2") action { (x, options) => options.copy(filter2 = x) - } text ("String whose presence indicates a datum for the secondary item set, can be a regex (optional). Used in cross-cooccurrence. Default: no secondary filter is applied") + } text ("String (or regex) whose presence indicates a datum for the secondary item set (optional). If not present no secondary dataset is collected.") opt[Int]("rowIDPosition") abbr ("rc") action { (x, options) => options.copy(rowIDPosition = x) @@ -126,33 +128,33 @@ object ItemSimilarityDriver extends MahoutDriver { if (x >= -1) success else failure("Option --filterColNum must be >= -1") } - note("\nDefault input schema will accept: 'userIDitemId' or 'userIDitemIDany-text...' and all rows will be used") + note("\nUsing all defaults the input is expected of the form: \"userIDitemId\" or \"userIDitemIDany-text...\" and all rows will be used") //File finding strategy--not driver specific note("\nFile input options:") opt[Unit]('r', "recursive") action { (_, options) => options.copy(recursive = true) - } text ("The input path should be searched recursively for files that match the filename pattern from -fp (optional), Default: false") + } text ("Searched the -i path recursively for files that match --filenamePattern (optional), Default: false") opt[String]("filenamePattern") abbr ("fp") action { (x, options) => options.copy(filenamePattern = x) - } text ("Regex to match in determining input files (optional). Default: filename in the --input option or '^part-.*' if --input is a directory") + } text ("Regex to match in determining input files (optional). Default: filename in the --input option or \"^part-.*\" if --input is a directory") //Drm output schema--not driver specific, drm specific note("\nOutput text file schema options:") opt[String]("rowKeyDelim") abbr ("rd") action { (x, options) => options.copy(rowKeyDelim = x) - } text ("Separates the rowID key from the vector values list (optional). Default: '\\t'") + } text ("Separates the rowID key from the vector values list (optional). Default: \"\\t\"") opt[String]("columnIdStrengthDelim") abbr ("cd") action { (x, options) => options.copy(columnIdStrengthDelim = x) - } text ("Separates column IDs from their values in the vector values list (optional). Default: ':'") + } text ("Separates column IDs from their values in the vector values list (optional). Default: \":\"") opt[String]("tupleDelim") abbr ("td") action { (x, options) => options.copy(tupleDelim = x) - } text ("Separates vector tuple values in the values list (optional). Default: ','") + } text ("Separates vector tuple values in the values list (optional). Default: \",\"") - note("\nDefault delimiters will produce output of the form: 'itemID1itemID2:value2,itemID10:value10...'") + note("\nDefault delimiters will produce output of the form: \"itemID1itemID2:value2,itemID10:value10...\"") //Driver notes--driver specific note("\nNote: Only the Log Likelihood Ratio (LLR) is supported as a similarity measure.\n") @@ -275,7 +277,7 @@ object ItemSimilarityDriver extends MahoutDriver { stop } - // Default values go here, any '_' or null should be 'required' in the Parser or flags an unused option + // Default values go here, any "_" or null should be "required" in the Parser or flags an unused option // todo: support two input streams for cross-similarity, maybe assume one schema for all inputs case class Options( master: String = "local", @@ -293,7 +295,7 @@ object ItemSimilarityDriver extends MahoutDriver { filterPosition: Int = -1, filter1: String = null, filter2: String = null, - inDelim: String = ",", + inDelim: String = "[,\t]", rowKeyDelim: String = "\t", columnIdStrengthDelim: String = ":", tupleDelim: String = ",") From 213b18dee259925de82c703451bdea640e1f068e Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 26 Jun 2014 15:26:17 -0700 Subject: [PATCH 15/38] added a job.xml assembly for creation of an all-dependencies jar --- spark/src/main/assembly/job.xml | 46 +++++++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 spark/src/main/assembly/job.xml diff --git a/spark/src/main/assembly/job.xml b/spark/src/main/assembly/job.xml new file mode 100644 index 0000000000..0c41f3d3ae --- /dev/null +++ b/spark/src/main/assembly/job.xml @@ -0,0 +1,46 @@ + + + + job + + jar + + false + + + true + + + + META-INF/LICENSE + + + runtime + / + true + + org.apache.hadoop:hadoop-core + + + + + \ No newline at end of file From 627d39f30860e4ab43783c72cc2cf8926060b73c Mon Sep 17 00:00:00 2001 From: pferrel Date: Fri, 27 Jun 2014 09:44:37 -0700 Subject: [PATCH 16/38] registered HashBiMap with JavaSerializer in Kryo --- .../mahout/sparkbindings/io/MahoutKryoRegistrator.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/mahout/sparkbindings/io/MahoutKryoRegistrator.scala b/spark/src/main/scala/org/apache/mahout/sparkbindings/io/MahoutKryoRegistrator.scala index b0042c93f0..79c758501e 100644 --- a/spark/src/main/scala/org/apache/mahout/sparkbindings/io/MahoutKryoRegistrator.scala +++ b/spark/src/main/scala/org/apache/mahout/sparkbindings/io/MahoutKryoRegistrator.scala @@ -18,6 +18,8 @@ package org.apache.mahout.sparkbindings.io import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.serializers.JavaSerializer +import com.google.common.collect.HashBiMap import org.apache.mahout.math._ import org.apache.spark.serializer.KryoRegistrator import org.apache.mahout.sparkbindings._ @@ -33,7 +35,7 @@ class MahoutKryoRegistrator extends KryoRegistrator { kryo.addDefaultSerializer(classOf[Vector], new WritableKryoSerializer[Vector, VectorWritable]) kryo.addDefaultSerializer(classOf[DenseVector], new WritableKryoSerializer[Vector, VectorWritable]) kryo.addDefaultSerializer(classOf[Matrix], new WritableKryoSerializer[Matrix, MatrixWritable]) - + kryo.register(classOf[com.google.common.collect.HashBiMap[String, Int]], new JavaSerializer()); } } From c273dc7de3c740189ce8157b334c2eef3a4c23ea Mon Sep 17 00:00:00 2001 From: pferrel Date: Fri, 27 Jun 2014 14:30:13 -0700 Subject: [PATCH 17/38] increased the default max heep for mahout/JVM to 4g, using max of 4g for Spark executor --- bin/mahout | 4 ++-- .../org/apache/mahout/drivers/ItemSimilarityDriver.scala | 6 +++--- .../apache/mahout/drivers/TextDelimitedReaderWriter.scala | 7 ++++++- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/bin/mahout b/bin/mahout index d312cbc2e4..56a3d0e451 100755 --- a/bin/mahout +++ b/bin/mahout @@ -7,7 +7,7 @@ # MAHOUT_JAVA_HOME The java implementation to use. Overrides JAVA_HOME. # # MAHOUT_HEAPSIZE The maximum amount of heap to use, in MB. -# Default is 1000. +# Default is 4000. # # HADOOP_CONF_DIR The location of a hadoop config directory # @@ -108,7 +108,7 @@ if [ "$JAVA_HOME" = "" ]; then fi JAVA=$JAVA_HOME/bin/java -JAVA_HEAP_MAX=-Xmx3g +JAVA_HEAP_MAX=-Xmx4g # check envvars which might override default args if [ "$MAHOUT_HEAPSIZE" != "" ]; then diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index fa0bb79337..4e5bc8c5c0 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -190,8 +190,8 @@ object ItemSimilarityDriver extends MahoutDriver { Unit = { sparkConf.set("spark.kryo.referenceTracking", "false") - .set("spark.kryoserializer.buffer.mb", "100") - .set("spark.executor.memory", "2g") + .set("spark.kryoserializer.buffer.mb", "200") + .set("spark.executor.memory", "4g") super.start(masterUrl, appName) @@ -295,7 +295,7 @@ object ItemSimilarityDriver extends MahoutDriver { filterPosition: Int = -1, filter1: String = null, filter2: String = null, - inDelim: String = "[,\t]", + inDelim: String = "[,\t, ]", rowKeyDelim: String = "\t", columnIdStrengthDelim: String = ":", tupleDelim: String = ",") diff --git a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala index ecaba6bf24..a31b772ab7 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala @@ -57,10 +57,15 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ if(filterPosition != -1) { // get the rows that have a column matching the filter columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) + }else{ + columns = columns.filter({ tokens => true}) } // get row and column IDs - val interactions = columns.map({ tokens => tokens(rowIDPosition) -> tokens(columnIDPosition)}) + //columns.collect + val interactions = columns.map{ tokens => + tokens(rowIDPosition) -> tokens(columnIDPosition) + } interactions.cache() From 9dd2f2eabf1bf64660de6b5b5e49aafe18229a7a Mon Sep 17 00:00:00 2001 From: Pat Ferrel Date: Mon, 30 Jun 2014 10:06:49 -0700 Subject: [PATCH 18/38] tweaking memory requirements to process epinions with the ItemSimilarityDriver --- .../scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 4e5bc8c5c0..fd9bb8a746 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -191,7 +191,7 @@ object ItemSimilarityDriver extends MahoutDriver { sparkConf.set("spark.kryo.referenceTracking", "false") .set("spark.kryoserializer.buffer.mb", "200") - .set("spark.executor.memory", "4g") + .set("spark.executor.memory", "6g") super.start(masterUrl, appName) From 6ec98f32775c791ee001fc996f475215e427f368 Mon Sep 17 00:00:00 2001 From: pferrel Date: Mon, 30 Jun 2014 10:08:49 -0700 Subject: [PATCH 19/38] refactored to use a DistributedContext instead of raw SparkContext --- .../apache/mahout/drivers/IndexedDataset.scala | 2 +- .../mahout/drivers/ItemSimilarityDriver.scala | 1 - .../apache/mahout/drivers/MahoutDriver.scala | 5 +++-- .../apache/mahout/drivers/ReaderWriter.scala | 17 +++++------------ .../org/apache/mahout/drivers/Schema.scala | 3 ++- .../drivers/TextDelimitedReaderWriter.scala | 17 ++++++++--------- 6 files changed, 19 insertions(+), 26 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala index 0903750e61..0d8c160bd3 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala @@ -18,7 +18,7 @@ package org.apache.mahout.drivers import com.google.common.collect.BiMap -import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} +import org.apache.mahout.math.drm.CheckpointedDrm /** * Wraps a [[org.apache.mahout.sparkbindings.drm.DrmLike]] object with two [[com.google.common.collect.BiMap]]s to store ID/label translation dictionaries. diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 4e5bc8c5c0..40d5234f3d 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -34,7 +34,6 @@ import org.apache.mahout.cf.CooccurrenceAnalysis */ object ItemSimilarityDriver extends MahoutDriver { //todo: Should also take two input streams and do cross similarity with no filter required. - //todo: Required: should work on the old Mahout TDF that allows values (but ignores them), // required for examples private var options: Options = _ diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala index 6af7696ab5..35d20d72b1 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala @@ -17,7 +17,8 @@ package org.apache.mahout.drivers -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.mahout.math.drm.DistributedContext +import org.apache.spark.SparkConf import org.apache.mahout.sparkbindings._ /** Extend this class to create a Mahout CLI driver. Minimally you must override process and main. @@ -50,7 +51,7 @@ import org.apache.mahout.sparkbindings._ * }}} */ abstract class MahoutDriver { - implicit var mc: SparkContext = _ + implicit var mc: DistributedContext = _ implicit val sparkConf = new SparkConf() /** Creates a Spark context to run the job inside. diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala index ae0fea8d7b..c5b7385f50 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ReaderWriter.scala @@ -17,23 +17,16 @@ package org.apache.mahout.drivers -import org.apache.spark.SparkContext._ -import org.apache.mahout.math.RandomAccessSparseVector -import org.apache.spark.SparkContext -import com.google.common.collect.{BiMap, HashBiMap} -import scala.collection.JavaConversions._ -import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} -import org.apache.mahout.sparkbindings._ - +import org.apache.mahout.math.drm.DistributedContext /** Reader trait is abstract in the sense that the reader function must be defined by an extending trait, which also defines the type to be read. * @tparam T type of object read, usually supplied by an extending trait. * @todo the reader need not create both dictionaries but does at present. There are cases where one or the other dictionary is never used so saving the memory for a very large dictionary may be worth the optimization to specify which dictionaries are created. */ trait Reader[T]{ - val mc: SparkContext + val mc: DistributedContext val readSchema: Schema - protected def reader(mc: SparkContext, readSchema: Schema, source: String): T + protected def reader(mc: DistributedContext, readSchema: Schema, source: String): T def readFrom(source: String): T = reader(mc, readSchema, source) } @@ -41,8 +34,8 @@ trait Reader[T]{ * @tparam T */ trait Writer[T]{ - val mc: SparkContext + val mc: DistributedContext val writeSchema: Schema - protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, collection: T): Unit + protected def writer(mc: DistributedContext, writeSchema: Schema, dest: String, collection: T): Unit def writeTo(collection: T, dest: String) = writer(mc, writeSchema, dest, collection) } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala b/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala index 50302a6bfa..46e1540ee5 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/Schema.scala @@ -17,6 +17,7 @@ package org.apache.mahout.drivers +import scala.collection.mutable import scala.collection.mutable.HashMap /** Syntactic sugar for HashMap[String, Any] @@ -24,6 +25,6 @@ import scala.collection.mutable.HashMap * @param params list of mappings for instantiation {{{val mySchema = new Schema("one" -> 1, "two" -> "2", ...)}}} */ class Schema(params: Tuple2[String, Any]*) extends HashMap[String, Any] { - // todo: this require a mutable HashMap, do we care? + // note: this require a mutable HashMap, do we care? this ++= params } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala index a31b772ab7..75f8b6e6f7 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala @@ -20,10 +20,9 @@ package org.apache.mahout.drivers import scala.collection.JavaConversions._ import org.apache.spark.SparkContext._ import org.apache.mahout.math.RandomAccessSparseVector -import org.apache.spark.SparkContext import com.google.common.collect.{BiMap, HashBiMap} import scala.collection.JavaConversions._ -import org.apache.mahout.math.drm.{CheckpointedDrm, DrmLike} +import org.apache.mahout.math.drm.{DistributedContext, CheckpointedDrm} import org.apache.mahout.sparkbindings._ @@ -37,7 +36,7 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ * @param source comma delimited URIs of text files to be read into the [[org.apache.mahout.drivers.IndexedDataset]] * @return */ - protected def reader(mc: SparkContext, readSchema: Schema, source: String): IndexedDataset = { + protected def reader(mc: DistributedContext, readSchema: Schema, source: String): IndexedDataset = { try { val delimiter = readSchema("delim").asInstanceOf[String] val rowIDPosition = readSchema("rowIDPosition").asInstanceOf[Int] @@ -134,7 +133,7 @@ trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ * @param writeSchema describes the delimiters and positions of values in the output text delimited file. * @param dest directory to write text delimited version of [[org.apache.mahout.drivers.IndexedDataset]] */ - protected def writer(mc: SparkContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { + protected def writer(mc: DistributedContext, writeSchema: Schema, dest: String, indexedDataset: IndexedDataset): Unit = { try { val rowKeyDelim = writeSchema("rowKeyDelim").asInstanceOf[String] val columnIdStrengthDelim = writeSchema("columnIdStrengthDelim").asInstanceOf[String] @@ -176,21 +175,21 @@ trait TDIndexedDatasetReaderWriter extends TDIndexedDatasetReader with TDIndexed * @param mc Spark context for reading files * @note The source is supplied by Reader#readFrom . * */ -class TextDelimitedIndexedDatasetReader(val readSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReader +class TextDelimitedIndexedDatasetReader(val readSchema: Schema)(implicit val mc: DistributedContext) extends TDIndexedDatasetReader /** Writes text delimited files into an IndexedDataset. Classes are needed to supply trait params in their constructor. * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. * @param mc Spark context for reading files * @note the destination is supplied by Writer#writeTo trait method * */ -class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetWriter +class TextDelimitedIndexedDatasetWriter(val writeSchema: Schema)(implicit val mc: DistributedContext) extends TDIndexedDatasetWriter /** Reads and writes text delimited files to/from an IndexedDataset. Classes are needed to supply trait params in their constructor. * @param readSchema describes the delimiters and position of values in the text delimited file(s) to be read. * @param writeSchema describes the delimiters and position of values in the text delimited file(s) written. * @param mc Spark context for reading the files, may be implicitly defined. * */ -class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema)(implicit val mc: SparkContext) extends TDIndexedDatasetReaderWriter +class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeSchema: Schema)(implicit val mc: DistributedContext) extends TDIndexedDatasetReaderWriter /** A version of IndexedDataset that has it's own writeTo method from a Writer trait. This is an alternative to creating * a Writer based stand-alone class for writing. Consider it experimental allowing similar semantics to drm.writeDrm(). @@ -200,10 +199,10 @@ class TextDelimitedIndexedDatasetReaderWriter(val readSchema: Schema, val writeS * @param rowIDs bi-directional dictionary for rows of external IDs to internal ordinal Mahout IDs. * @param columnIDs bi-directional dictionary for columns of external IDs to internal ordinal Mahout IDs. * @param writeSchema contains params for the schema/format or the written text delimited file. - * @param mc mahout distributed context (SparkContext) may be implicitly defined. + * @param mc mahout distributed context (DistributedContext) may be implicitly defined. * */ class IndexedDatasetTextDelimitedWriteable(matrix: CheckpointedDrm[Int], rowIDs: BiMap[String,Int], columnIDs: BiMap[String,Int], - val writeSchema: Schema)(implicit val mc: SparkContext) + val writeSchema: Schema)(implicit val mc: DistributedContext) extends IndexedDataset(matrix, rowIDs, columnIDs) with TDIndexedDatasetWriter { def writeTo(dest: String): Unit = { From 8e70091a564c8464ea70bf90006d8124c3a7f208 Mon Sep 17 00:00:00 2001 From: pferrel Date: Mon, 30 Jun 2014 13:11:42 -0700 Subject: [PATCH 20/38] fixed a bug, SparkConf in driver was ignored and blank one passed in to create a DistributedContext --- .../org/apache/mahout/drivers/ItemSimilarityDriver.scala | 2 +- .../main/scala/org/apache/mahout/drivers/MahoutDriver.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 203010537c..5503239a64 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -190,7 +190,7 @@ object ItemSimilarityDriver extends MahoutDriver { sparkConf.set("spark.kryo.referenceTracking", "false") .set("spark.kryoserializer.buffer.mb", "200") - .set("spark.executor.memory", "6g") + .set("spark.executor.memory", "5g") super.start(masterUrl, appName) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala index 35d20d72b1..49cc54f27b 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala @@ -63,12 +63,12 @@ abstract class MahoutDriver { * */ protected def start(masterUrl: String, appName: String, customJars:Traversable[String] = Traversable.empty[String]) : Unit = { - mc = mahoutSparkContext(masterUrl, appName, customJars) + mc = mahoutSparkContext(masterUrl, appName, customJars, sparkConf) } /** Override (optionally) for special cleanup */ protected def stop: Unit = { - mc.stop + mc.close } /** This is wher you do the work, call start first, then before exiting call stop */ From 01a0341f56071d2244aabd6de8c6f528ad35b164 Mon Sep 17 00:00:00 2001 From: pferrel Date: Mon, 30 Jun 2014 13:33:39 -0700 Subject: [PATCH 21/38] added option for configuring Spark executor memory --- .../org/apache/mahout/drivers/ItemSimilarityDriver.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 5503239a64..10894579a6 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -153,6 +153,12 @@ object ItemSimilarityDriver extends MahoutDriver { options.copy(tupleDelim = x) } text ("Separates vector tuple values in the values list (optional). Default: \",\"") + //Spark config options--not driver specific + note("\nSpark config options:") + opt[String]("sparkExecutorMem") abbr ("sem") action { (x, options) => + options.copy(sparkExecutorMem = x) + } text ("Max Java heap available as \"executor memory\" on each node (optional). Default: 4g") + note("\nDefault delimiters will produce output of the form: \"itemID1itemID2:value2,itemID10:value10...\"") //Driver notes--driver specific @@ -190,7 +196,7 @@ object ItemSimilarityDriver extends MahoutDriver { sparkConf.set("spark.kryo.referenceTracking", "false") .set("spark.kryoserializer.buffer.mb", "200") - .set("spark.executor.memory", "5g") + .set("spark.executor.memory", options.sparkExecutorMem) super.start(masterUrl, appName) @@ -280,6 +286,7 @@ object ItemSimilarityDriver extends MahoutDriver { // todo: support two input streams for cross-similarity, maybe assume one schema for all inputs case class Options( master: String = "local", + sparkExecutorMem: String = "4g", appName: String = "ItemSimilarityJob", randomSeed: Int = System.currentTimeMillis().toInt, recursive: Boolean = false, From 2d9efd73def8207dded5cd1dd8699035a8cc1b34 Mon Sep 17 00:00:00 2001 From: pferrel Date: Mon, 30 Jun 2014 15:37:19 -0700 Subject: [PATCH 22/38] removed some outdated examples --- .../mahout/cf/examples/Recommendations.scala | 172 ------------------ 1 file changed, 172 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala diff --git a/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala b/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala deleted file mode 100644 index afd6701605..0000000000 --- a/spark/src/main/scala/org/apache/mahout/cf/examples/Recommendations.scala +++ /dev/null @@ -1,172 +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.mahout.cf.examples - -import scala.io.Source -import org.apache.mahout.math._ -import scalabindings._ -import RLikeOps._ -import drm._ -import RLikeDrmOps._ -import org.apache.mahout.sparkbindings._ - -import org.apache.mahout.cf.CooccurrenceAnalysis._ -import scala.collection.JavaConversions._ - -/** - * The Epinions dataset contains ratings from users to items and a trust-network between the users. - * We use co-occurrence analysis to compute "users who like these items, also like that items" and - * "users who trust these users, like that items" - * - * Download and unpack the dataset files from: - * - * http://www.trustlet.org/datasets/downloaded_epinions/ratings_data.txt.bz2 - * http://www.trustlet.org/datasets/downloaded_epinions/trust_data.txt.bz2 - **/ -object RunCrossCooccurrenceAnalysisOnEpinions { - - def main(args: Array[String]): Unit = { - - if (args.length == 0) { - println("Usage: RunCooccurrenceAnalysisOnMovielens1M ") - println("Download the dataset from http://www.trustlet.org/datasets/downloaded_epinions/ratings_data.txt.bz2 and") - println("http://www.trustlet.org/datasets/downloaded_epinions/trust_data.txt.bz2") - sys.exit(-1) - } - - val datasetDir = args(0) - - val epinionsRatings = new SparseMatrix(49290, 139738) - - var firstLineSkipped = false - for (line <- Source.fromFile(datasetDir + "/ratings_data.txt").getLines()) { - if (line.contains(' ') && firstLineSkipped) { - val tokens = line.split(' ') - val userID = tokens(0).toInt - 1 - val itemID = tokens(1).toInt - 1 - val rating = tokens(2).toDouble - epinionsRatings(userID, itemID) = rating - } - firstLineSkipped = true - } - - val epinionsTrustNetwork = new SparseMatrix(49290, 49290) - firstLineSkipped = false - for (line <- Source.fromFile(datasetDir + "/trust_data.txt").getLines()) { - if (line.contains(' ') && firstLineSkipped) { - val tokens = line.trim.split(' ') - val userID = tokens(0).toInt - 1 - val trustedUserId = tokens(1).toInt - 1 - epinionsTrustNetwork(userID, trustedUserId) = 1 - } - firstLineSkipped = true - } - - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "100") -/* to run on local, can provide number of core by changing to local[4] */ - implicit val distributedContext = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", - customJars = Traversable.empty[String]) - - /* to run on a Spark cluster provide the Spark Master URL - implicit val distributedContext = mahoutSparkContext(masterUrl = "spark://occam4:7077", appName = "MahoutClusteredContext", - customJars = Traversable.empty[String]) -*/ - val drmEpinionsRatings = drmParallelize(epinionsRatings, numPartitions = 2) - val drmEpinionsTrustNetwork = drmParallelize(epinionsTrustNetwork, numPartitions = 2) - - val indicatorMatrices = cooccurrences(drmEpinionsRatings, randomSeed = 0xdeadbeef, - maxInterestingItemsPerThing = 100, maxNumInteractions = 500, Array(drmEpinionsTrustNetwork)) - -/* local storage */ - RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(0), - "/tmp/co-occurrence-on-epinions/indicators-item-item/") - RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(1), - "/tmp/co-occurrence-on-epinions/indicators-trust-item/") - -/* To run on HDFS put your path to the data here, example of fully qualified path on my cluster provided - RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(0), - "hdfs://occam4:54310/user/pat/xrsj/indicators-item-item/") - RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrices(1), - "hdfs://occam4:54310/user/pat/xrsj/indicators-trust-item/") -*/ - distributedContext.close() - - println("Saved indicators to /tmp/co-occurrence-on-epinions/") - } -} - -/** - * The movielens1M dataset contains movie ratings, we use co-occurrence analysis to compute - * "users who like these movies, also like that movies" - * - * Download and unpack the dataset files from: - * http://files.grouplens.org/datasets/movielens/ml-1m.zip - */ -object RunCooccurrenceAnalysisOnMovielens1M { - - def main(args: Array[String]): Unit = { - - if (args.length == 0) { - println("Usage: RunCooccurrenceAnalysisOnMovielens1M ") - println("Download the dataset from http://files.grouplens.org/datasets/movielens/ml-1m.zip") - sys.exit(-1) - } - - val datasetDir = args(0) - - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "100") - - implicit val sc = mahoutSparkContext(masterUrl = "local", appName = "MahoutLocalContext", - customJars = Traversable.empty[String]) - - System.setProperty("mahout.math.AtA.maxInMemNCol", 4000.toString) - - val movielens = new SparseMatrix(6040, 3952) - - for (line <- Source.fromFile(datasetDir + "/ratings.dat").getLines()) { - val tokens = line.split("::") - val userID = tokens(0).toInt - 1 - val itemID = tokens(1).toInt - 1 - val rating = tokens(2).toDouble - movielens(userID, itemID) = rating - } - - val drmMovielens = drmParallelize(movielens, numPartitions = 2) - - val indicatorMatrix = cooccurrences(drmMovielens).head - - RecommendationExamplesHelper.saveIndicatorMatrix(indicatorMatrix, - "/tmp/co-occurrence-on-movielens/indicators-item-item/") - - sc.stop() - - println("Saved indicators to /tmp/co-occurrence-on-movielens/") - } -} - -object RecommendationExamplesHelper { - - def saveIndicatorMatrix(indicatorMatrix: DrmLike[Int], path: String) = { - indicatorMatrix.rdd.flatMap({ case (thingID, itemVector) => - for (elem <- itemVector.nonZeroes()) yield { thingID + '\t' + elem.index } - }) - .saveAsTextFile(path) - } -} From 9fb281022cba7666dd26701b3d97d200b13c35f8 Mon Sep 17 00:00:00 2001 From: pferrel Date: Tue, 1 Jul 2014 11:17:42 -0700 Subject: [PATCH 23/38] test naming and pom changed to up the jvm heap max to 512m for scalatests --- spark/pom.xml | 1 + .../mahout/drivers/ItemSimilarityDriver.scala | 4 ++-- .../drivers/TextDelimitedReaderWriter.scala | 14 +++++------- .../drivers/ItemSimilarityDriverSuite.scala | 22 +++++++++---------- 4 files changed, 20 insertions(+), 21 deletions(-) diff --git a/spark/pom.xml b/spark/pom.xml index daa4aeceb4..602f693253 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -166,6 +166,7 @@ ${project.build.directory}/scalatest-reports . WDF TestSuite.txt + -Xmx512m diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 10894579a6..6943352718 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -286,7 +286,7 @@ object ItemSimilarityDriver extends MahoutDriver { // todo: support two input streams for cross-similarity, maybe assume one schema for all inputs case class Options( master: String = "local", - sparkExecutorMem: String = "4g", + sparkExecutorMem: String = "2g", appName: String = "ItemSimilarityJob", randomSeed: Int = System.currentTimeMillis().toInt, recursive: Boolean = false, @@ -301,7 +301,7 @@ object ItemSimilarityDriver extends MahoutDriver { filterPosition: Int = -1, filter1: String = null, filter2: String = null, - inDelim: String = "[,\t, ]", + inDelim: String = "[,\t ]", rowKeyDelim: String = "\t", columnIdStrengthDelim: String = ":", tupleDelim: String = ",") diff --git a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala index 75f8b6e6f7..119f8d3680 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala @@ -50,27 +50,25 @@ trait TDIndexedDatasetReader extends Reader[IndexedDataset]{ throw new IllegalArgumentException }) - var columns = mc.textFile(source).map({ line => line.split(delimiter)}) + var columns = mc.textFile(source).map { line => line.split(delimiter) } // -1 means no filter in the input text, take them all if(filterPosition != -1) { // get the rows that have a column matching the filter - columns = columns.filter({ tokens => tokens(filterPosition) == filterBy}) - }else{ - columns = columns.filter({ tokens => true}) + columns = columns.filter { tokens => tokens(filterPosition) == filterBy } } // get row and column IDs - //columns.collect - val interactions = columns.map{ tokens => + val m = columns.collect + val interactions = columns.map { tokens => tokens(rowIDPosition) -> tokens(columnIDPosition) } interactions.cache() // create separate collections of rowID and columnID tokens - val rowIDs = interactions.map { case (rowID, _) => rowID}.distinct().collect() - val columnIDs = interactions.map { case (_, columnID) => columnID}.distinct().collect() + val rowIDs = interactions.map { case (rowID, _) => rowID }.distinct().collect() + val columnIDs = interactions.map { case (_, columnID) => columnID }.distinct().collect() val numRows = rowIDs.size val numColumns = columnIDs.size diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index 3e7f4c35d5..3a0410f1ff 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -71,10 +71,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc )) */ - test ("running simple, non-full-spec CSV through"){ + test ("ItemSimilarityDriver, non-full-spec CSV"){ - val InFile = TmpDir + "in-file.csv" - val OutPath = TmpDir + "indicator-matrices" + val InFile = TmpDir + "in-file.csv/" //using part files, not singel file + val OutPath = TmpDir + "indicator-matrices/" val lines = Array( "u1,purchase,iphone", @@ -125,10 +125,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc - test ("Running TSV files through"){ + test ("ItemSimilarityDriver TSV "){ - val InFile = TmpDir + "in-file.tsv" - val OutPath = TmpDir + "indicator-matrices" + val InFile = TmpDir + "in-file.tsv/" + val OutPath = TmpDir + "indicator-matrices/" val lines = Array( "u1\tpurchase\tiphone", @@ -178,10 +178,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } - test ("Running log files through"){ + test ("ItemSimilarityDriver log-ish files"){ - val InFile = TmpDir + "in-file.log" - val OutPath = TmpDir + "indicator-matrices" + val InFile = TmpDir + "in-file.log/" + val OutPath = TmpDir + "indicator-matrices/" val lines = Array( "2014-06-23 14:46:53.115\tu1\tpurchase\trandom text\tiphone", @@ -231,7 +231,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } - test ("Running legacy files through"){ + test ("ItemSimilarityDriver legacy supported file format"){ val InDir = TmpDir + "in-dir/" val InFilename = "in-file.tsv" @@ -277,7 +277,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } - test("recursive file discovery using filename patterns"){ + test("ItemSimilarityDriver recursive file discovery using filename patterns"){ //directory structure using the following // tmp/data/m1.tsv // tmp/data/more-data/another-dir/m2.tsv From 606fe9935fbd4398a45000e5668f40522be8c0e7 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 3 Jul 2014 10:39:19 -0700 Subject: [PATCH 24/38] found why mrlegacy itemsimilarity and spark version disagreed on value fixed spark --- .../mahout/cf/CooccurrenceAnalysis.scala | 23 +- .../mahout/drivers/ItemSimilarityDriver.scala | 13 +- .../apache/mahout/drivers/MahoutDriver.scala | 8 +- .../mahout/cf/CooccurrenceAnalysisSuite.scala | 200 +++++++++++------- .../drivers/ItemSimilarityDriverSuite.scala | 47 ++-- 5 files changed, 186 insertions(+), 105 deletions(-) rename {spark => math-scala}/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala (91%) diff --git a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala similarity index 91% rename from spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala rename to math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index ee44f90d3a..49f89f0893 100644 --- a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -22,7 +22,7 @@ import scalabindings._ import RLikeOps._ import drm._ import RLikeDrmOps._ -import org.apache.mahout.sparkbindings._ +//import org.apache.mahout.sparkbindings._ import scala.collection.JavaConversions._ import org.apache.mahout.math.stats.LogLikelihood import collection._ @@ -96,15 +96,29 @@ object CooccurrenceAnalysis extends Serializable { * Compute loglikelihood ratio * see http://tdunning.blogspot.de/2008/03/surprise-and-coincidence.html for details **/ - def loglikelihoodRatio(numInteractionsWithA: Long, numInteractionsWithB: Long, + def logLikelihoodRatio(numInteractionsWithA: Long, numInteractionsWithB: Long, numInteractionsWithAandB: Long, numInteractions: Long) = { val k11 = numInteractionsWithAandB val k12 = numInteractionsWithA - numInteractionsWithAandB val k21 = numInteractionsWithB - numInteractionsWithAandB val k22 = numInteractions - numInteractionsWithA - numInteractionsWithB + numInteractionsWithAandB +/* + long k11 = (long) element.get(); + long k12 = (long) (rowSums.get(row.index()) - k11); + long k21 = (long) (colSums.get(element.index()) - k11); + long k22 = (long) (total - k11 - k12 - k21); + double score = LogLikelihood.rootLogLikelihoodRatio(k11, k12, k21, k22); +*/ +/* + val k11 = numInteractionsWithAandB + val k12 = numInteractionsWithA - k11 + val k21 = numInteractionsWithB - k11 + val k22 = numInteractions - k11 - k12 - k21 +*/ LogLikelihood.logLikelihoodRatio(k11, k12, k21, k22) + } def computeIndicators(drmBtA: DrmLike[Int], numUsers: Int, maxInterestingItemsPerThing: Int, @@ -131,9 +145,10 @@ object CooccurrenceAnalysis extends Serializable { // exclude co-occurrences of the item with itself if (crossCooccurrence || thingB != thingA) { // Compute loglikelihood ratio - val llrRatio = loglikelihoodRatio(numInteractionsB(thingB).toLong, numInteractionsA(thingA).toLong, + val llrRatio = logLikelihoodRatio(numInteractionsB(thingB).toLong, numInteractionsA(thingA).toLong, cooccurrences.toLong, numUsers) - val candidate = thingA -> llrRatio + val tLLR = 1.0 - (1.0 / (1.0 + llrRatio)) + val candidate = thingA -> tLLR // Enqueue item with score, if belonging to the top-k if (topItemsPerThing.size < maxInterestingItemsPerThing) { diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 6943352718..77005f1ecd 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -21,8 +21,7 @@ import org.apache.mahout.cf.CooccurrenceAnalysis /** * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. - * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. - * It reads text lines + * Reads text lines * that contain (row id, column id, ...). The IDs are user specified strings which will be * preserved in the * output. The individual tuples will be accumulated into a matrix and [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]] @@ -31,6 +30,13 @@ import org.apache.mahout.cf.CooccurrenceAnalysis * similarity of the primary * to the secondary. Returns one or two directories of text files formatted as specified in * the options. + * The options allow flexible control of the input schema, file discovery, output schema, and control of + * algorithm parameters. + * To get help run {{{mahout spark-itemsimilarity}}} for a full explanation of options. To process simple + * tuples of text delimited values (userID,itemID) with or without a strengths and with a separator of tab, comma, or space, + * you can specify only the input and output file and directory--all else will default to the correct values. + * @note To use with a Spark cluster see the --masterUrl option, if you run out of heap space check + * the --sparkExecutorMemory option. */ object ItemSimilarityDriver extends MahoutDriver { //todo: Should also take two input streams and do cross similarity with no filter required. @@ -190,8 +196,7 @@ object ItemSimilarityDriver extends MahoutDriver { } override def start(masterUrl: String = options.master, - appName: String = options.appName, - customJars:Traversable[String] = Traversable.empty[String]): + appName: String = options.appName): Unit = { sparkConf.set("spark.kryo.referenceTracking", "false") diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala index 49cc54f27b..afc7c1e997 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala @@ -61,8 +61,12 @@ abstract class MahoutDriver { * @param appName Name to display in Spark UI * @param customJars List of paths to custom jars * */ - protected def start(masterUrl: String, appName: String, - customJars:Traversable[String] = Traversable.empty[String]) : Unit = { + protected def start(masterUrl: String, appName: String, customJars:Traversable[String]) : Unit = { + mc = mahoutSparkContext(masterUrl, appName, customJars, sparkConf) + } + + protected def start(masterUrl: String, appName: String) : Unit = { + val customJars = Traversable.empty[String] mc = mahoutSparkContext(masterUrl, appName, customJars, sparkConf) } diff --git a/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala b/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala index 2db5f50c3c..e46dad52bc 100644 --- a/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala @@ -17,13 +17,11 @@ package org.apache.mahout.cf -import org.scalatest.FunSuite -import org.apache.mahout.test.MahoutSuite -import org.apache.mahout.math.scalabindings._ -import org.apache.mahout.math.scalabindings.MatrixOps import org.apache.mahout.math.drm._ -import org.apache.mahout.math._ +import org.apache.mahout.math.scalabindings.{MatrixOps, _} import org.apache.mahout.sparkbindings.test.MahoutLocalContext +import org.apache.mahout.test.MahoutSuite +import org.scalatest.FunSuite /* values A = @@ -41,32 +39,42 @@ B = class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLocalContext { + // correct cooccurrence with LLR + final val matrixLLRCoocAtAControl = dense( + (0.0, 0.6331745808516107, 0.0, 0.0, 0.0), + (0.6331745808516107, 0.0, 0.0, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.6331745808516107, 0.0), + (0.0, 0.0, 0.6331745808516107, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.0, 0.0)) + + // correct cross-cooccurrence with LLR + final val matrixLLRCoocBtAControl = dense( + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.40461878191490940), + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), + (0.0, 0.0, 0.0, 0.0, 0.8181382096075936)) + + + test("cooccurrence [A'A], [B'A] boolbean data using LLR") { - val a = dense((1, 1, 0, 0, 0), (0, 0, 1, 1, 0), (0, 0, 0, 0, 1), (1, 0, 0, 1, 0)) - val b = dense((1, 1, 1, 1, 0), (1, 1, 1, 1, 0), (0, 0, 1, 0, 1), (1, 1, 0, 1, 0)) + val a = dense( + (1, 1, 0, 0, 0), + (0, 0, 1, 1, 0), + (0, 0, 0, 0, 1), + (1, 0, 0, 1, 0)) + + val b = dense( + (1, 1, 1, 1, 0), + (1, 1, 1, 1, 0), + (0, 0, 1, 0, 1), + (1, 1, 0, 1, 0)) + val drmA = drmParallelize(m = a, numPartitions = 2) val drmB = drmParallelize(m = b, numPartitions = 2) - // correct cooccurrence with LLR - val matrixLLRCoocAtAControl = dense( - (0.0, 1.7260924347106847, 0, 0, 0), - (1.7260924347106847, 0, 0, 0, 0), - (0, 0, 0, 1.7260924347106847, 0), - (0, 0, 1.7260924347106847, 0, 0), - (0, 0, 0, 0, 0) - ) - - // correct cross-cooccurrence with LLR - val matrixLLRCoocBtAControl = dense( - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.6795961471815897), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (0, 0, 0, 0, 4.498681156950466) - ) - //self similarity - val drmCooc = CooccurrenceAnalysis.cooccurrences(drmARaw = drmA, drmBs = Array(drmB)) + val drmCooc = CooccurrenceAnalysis.cooccurrences(drmARaw = drmA, randomSeed = 1, drmBs = Array(drmB)) val matrixSelfCooc = drmCooc(0).checkpoint().collect val diffMatrix = matrixSelfCooc.minus(matrixLLRCoocAtAControl) var n = (new MatrixOps(m = diffMatrix)).norm @@ -77,32 +85,25 @@ class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLoc val diff2Matrix = matrixCrossCooc.minus(matrixLLRCoocBtAControl) n = (new MatrixOps(m = diff2Matrix)).norm n should be < 1E-10 + } test("cooccurrence [A'A], [B'A] double data using LLR") { - val a = dense((100000.0D, 1.0D, 0.0D, 0.0D, 0.0D), (0.0D, 0.0D, 10.0D, 1.0D, 0.0D), (0.0D, 0.0D, 0.0D, 0.0D, 1000.0D), (1.0D, 0.0D, 0.0D, 10.0D, 0.0D)) - val b = dense((10000.0D, 100.0D, 1000.0D, 1.0D, 0.0D), (10.0D, 1.0D, 10000000.0D, 10.0D, 0.0D), (0.0D, 0.0D, 1000.0D, 0.0D, 100.0D), (100.0D, 1.0D, 0.0D, 100000.0D, 0.0D)) + val a = dense( + (100000.0D, 1.0D, 0.0D, 0.0D, 0.0D), + (0.0D, 0.0D, 10.0D, 1.0D, 0.0D), + (0.0D, 0.0D, 0.0D, 0.0D, 1000.0D), + (1.0D, 0.0D, 0.0D, 10.0D, 0.0D)) + + val b = dense( + (10000.0D, 100.0D, 1000.0D, 1.0D, 0.0D), + (10.0D, 1.0D, 10000000.0D, 10.0D, 0.0D), + (0.0D, 0.0D, 1000.0D, 0.0D, 100.0D), + (100.0D, 1.0D, 0.0D, 100000.0D, 0.0D)) + val drmA = drmParallelize(m = a, numPartitions = 2) val drmB = drmParallelize(m = b, numPartitions = 2) - // correct cooccurrence with LLR - val matrixLLRCoocAtAControl = dense( - (0.0, 1.7260924347106847, 0, 0, 0), - (1.7260924347106847, 0, 0, 0, 0), - (0, 0, 0, 1.7260924347106847, 0), - (0, 0, 1.7260924347106847, 0, 0), - (0, 0, 0, 0, 0) - ) - - // correct cross-cooccurrence with LLR - val matrixLLRCoocBtAControl = dense( - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.6795961471815897), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (0, 0, 0, 0, 4.498681156950466) - ) - //self similarity val drmCooc = CooccurrenceAnalysis.cooccurrences(drmARaw = drmA, drmBs = Array(drmB)) val matrixSelfCooc = drmCooc(0).checkpoint().collect @@ -118,30 +119,22 @@ class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLoc } test("cooccurrence [A'A], [B'A] integer data using LLR") { - val a = dense((1000, 10, 0, 0, 0), (0, 0, -10000, 10, 0), (0, 0, 0, 0, 100), (10000, 0, 0, 1000, 0)) - val b = dense((100, 1000, -10000, 10000, 0), (10000, 1000, 100, 10, 0), (0, 0, 10, 0, -100), (10, 100, 0, 1000, 0)) + val a = dense( + (1000, 10, 0, 0, 0), + (0, 0, -10000, 10, 0), + (0, 0, 0, 0, 100), + (10000, 0, 0, 1000, 0)) + + val b = dense( + (100, 1000, -10000, 10000, 0), + (10000, 1000, 100, 10, 0), + (0, 0, 10, 0, -100), + (10, 100, 0, 1000, 0)) + val drmA = drmParallelize(m = a, numPartitions = 2) val drmB = drmParallelize(m = b, numPartitions = 2) - // correct cooccurrence with LLR - val matrixLLRCoocAtAControl = dense( - (0.0, 1.7260924347106847, 0, 0, 0), - (1.7260924347106847, 0, 0, 0, 0), - (0, 0, 0, 1.7260924347106847, 0), - (0, 0, 1.7260924347106847, 0, 0), - (0, 0, 0, 0, 0) - ) - - // correct cross-cooccurrence with LLR - val matrixLLRCoocBtAControl = dense( - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.6795961471815897), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0), - (0, 0, 0, 0, 4.498681156950466) - ) - - //self similarity + //self similarity val drmCooc = CooccurrenceAnalysis.cooccurrences(drmARaw = drmA, drmBs = Array(drmB)) //var cp = drmSelfCooc(0).checkpoint() //cp.writeDRM("/tmp/cooc-spark/")//to get values written @@ -158,22 +151,69 @@ class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLoc } test("LLR calc") { - val numInteractionsWithAandB = 10L - val numInteractionsWithA = 100L - val numInteractionsWithB = 200L - val numInteractions = 10000l - - val llr = CooccurrenceAnalysis.loglikelihoodRatio(numInteractionsWithA, numInteractionsWithB, numInteractionsWithAandB, numInteractions) - - assert(llr == 17.19462327013025) + val A = dense( + (1, 1, 0, 0, 0), + (0, 0, 1, 1, 0), + (0, 0, 0, 0, 1), + (1, 0, 0, 1, 0)) + + val AtA = A.transpose().times(A) + + /* AtA is: + 0 => {0:2.0,1:1.0,3:1.0} + 1 => {0:1.0,1:1.0} + 2 => {2:1.0,3:1.0} + 3 => {0:1.0,2:1.0,3:2.0} + 4 => {4:1.0} + + val AtAd = dense( + (2, 1, 0, 1, 0), + (1, 1, 0, 0, 0), + (0, 0, 1, 1, 0), + (1, 0, 1, 2, 0), + (0, 0, 0, 0, 1)) + + val AtAdNoSelfCooc = dense( + (0, 1, 0, 1, 0), + (1, 0, 0, 0, 0), + (0, 0, 0, 1, 0), + (1, 0, 1, 0, 0), + (0, 0, 0, 0, 0)) + + for (MatrixSlice row : cooccurrence) { + for (Vector.Element element : row.vector().nonZeroes()) { + long k11 = (long) element.get();// = 1 + long k12 = (long) (rowSums.get(row.index()) - k11);// = 0 + long k21 = (long) (colSums.get(element.index()) - k11);// = 1 + long k22 = (long) (total - k11 - k12 - k21);// = 2 + double score = LogLikelihood.rootLogLikelihoodRatio(k11, k12, k21, k22); + element.set(score); + } + } + + for some reason the hadoop version returns the following + return 1.0 - 1.0 / (1.0 + logLikelihood); + so not a pure llr or root llr + + */ + + //item (1,0) + val numInteractionsWithAandB = 1L + val numInteractionsWithA = 1L + val numInteractionsWithB = 2L + val numInteractions = 6l + + val llr = CooccurrenceAnalysis.logLikelihoodRatio(numInteractionsWithA, numInteractionsWithB, numInteractionsWithAandB, numInteractions) + + assert(llr == 2.6341457841558764) // value calculated by hadoop itemsimilairty } test("downsampling by number per row") { - val a = dense((1, 1, 1, 1, 0), - (1, 1, 1, 1, 1), - (0, 0, 0, 0, 1), - (1, 1, 0, 1, 0) - ) + val a = dense( + (1, 1, 1, 1, 0), + (1, 1, 1, 1, 1), + (0, 0, 0, 0, 1), + (1, 1, 0, 1, 0)) val drmA: DrmLike[Int] = drmParallelize(m = a, numPartitions = 2) val downSampledDrm = CooccurrenceAnalysis.sampleDownAndBinarize(drmA, 0xdeadbeef, 4) diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index 3a0410f1ff..f649d7bc12 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -26,19 +26,36 @@ import org.apache.mahout.test.MahoutSuite class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLocalContext { +/* + // correct self-cooccurrence with LLR + final val matrixLLRCoocAtAControl = dense( + (0.0, 0.6331745808516107, 0.0, 0.0, 0.0), + (0.6331745808516107, 0.0, 0.0, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.6331745808516107, 0.0), + (0.0, 0.0, 0.6331745808516107, 0.0, 0.0), + (0.0, 0.0, 0.0, 0.0, 0.0)) + + // correct cross-cooccurrence with LLR + final val matrixLLRCoocBtAControl = dense( + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.40461878191490940), + (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), + (0.0, 0.0, 0.0, 0.0, 0.8181382096075936)) +*/ + final val SelfSimilairtyTSV = Set( - "galaxy\tnexus:1.7260924347106847", - "ipad\tiphone:1.7260924347106847", - "nexus\tgalaxy:1.7260924347106847", - "iphone\tipad:1.7260924347106847", + "galaxy\tnexus:0.6331745808516107", + "ipad\tiphone:0.6331745808516107", + "nexus\tgalaxy:0.6331745808516107", + "iphone\tipad:0.6331745808516107", "surface") - - final val CrossSimilarityTSV = Set("" + - "nexus\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,surface:0.6795961471815897,galaxy:1.7260924347106847", - "ipad\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "surface\tsurface:4.498681156950466", - "iphone\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "galaxy\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847") + final val CrossSimilarityTSV = Set( + "galaxy\tnexus:0.4046187819149094,iphone:0.6331745808516107,ipad:0.4046187819149094,galaxy:0.6331745808516107", + "surface\tsurface:0.8181382096075936", + "nexus\tnexus:0.4046187819149094,iphone:0.6331745808516107,ipad:0.4046187819149094,surface:0.4046187819149094,galaxy:0.6331745808516107", + "ipad\tnexus:0.4046187819149094,iphone:0.6331745808516107,ipad:0.4046187819149094,galaxy:0.6331745808516107", + "iphone\tnexus:0.4046187819149094,iphone:0.6331745808516107,ipad:0.4046187819149094,galaxy:0.6331745808516107") final val TmpDir = "tmp/" // all IO going to whatever the default HDFS config is pointing to @@ -249,11 +266,11 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "3,3,1") val Answer = Set( - "0\t1:1.7260924347106847", - "3\t2:1.7260924347106847", - "1\t0:1.7260924347106847", + "0\t1:0.6331745808516107", + "3\t2:0.6331745808516107", + "1\t0:0.6331745808516107", "4", - "2\t3:1.7260924347106847") + "2\t3:0.6331745808516107") // this creates one part-0000 file in the directory mahoutCtx.parallelize(lines).coalesce(1, shuffle=true).saveAsTextFile(InDir) From 79b55ee515fe7c97b91b365e05e46366d41b1d7a Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 3 Jul 2014 11:21:42 -0700 Subject: [PATCH 25/38] added comment about the way llr is calculated --- .../org/apache/mahout/cf/CooccurrenceAnalysis.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index 49f89f0893..d4eee905d9 100644 --- a/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -19,15 +19,12 @@ package org.apache.mahout.cf import org.apache.mahout.math._ import scalabindings._ -import RLikeOps._ import drm._ import RLikeDrmOps._ -//import org.apache.mahout.sparkbindings._ import scala.collection.JavaConversions._ import org.apache.mahout.math.stats.LogLikelihood import collection._ import org.apache.mahout.common.RandomUtils -import org.apache.mahout.math.function.{VectorFunction, Functions} /** @@ -145,9 +142,12 @@ object CooccurrenceAnalysis extends Serializable { // exclude co-occurrences of the item with itself if (crossCooccurrence || thingB != thingA) { // Compute loglikelihood ratio - val llrRatio = logLikelihoodRatio(numInteractionsB(thingB).toLong, numInteractionsA(thingA).toLong, + val llr = logLikelihoodRatio(numInteractionsB(thingB).toLong, numInteractionsA(thingA).toLong, cooccurrences.toLong, numUsers) - val tLLR = 1.0 - (1.0 / (1.0 + llrRatio)) + + //not sure why this is calculated but it matches code in the hadoop version + val tLLR = 1.0 - (1.0 / (1.0 + llr)) + val candidate = thingA -> tLLR // Enqueue item with score, if belonging to the top-k From fd0931f87c57105d6b8bfefac5e6af27c26923ff Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 3 Jul 2014 12:30:17 -0700 Subject: [PATCH 26/38] moving cooc is not so easy because the tests seem to rely on MahoutLocalContext, need to suss that out first --- .../main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename {math-scala => spark}/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala (99%) diff --git a/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala similarity index 99% rename from math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala rename to spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index d4eee905d9..32dddcce71 100644 --- a/math-scala/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -147,7 +147,7 @@ object CooccurrenceAnalysis extends Serializable { //not sure why this is calculated but it matches code in the hadoop version val tLLR = 1.0 - (1.0 / (1.0 + llr)) - + val candidate = thingA -> tLLR // Enqueue item with score, if belonging to the top-k From e157137166e53b8517b83395f132ea43af8a135f Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 3 Jul 2014 13:17:19 -0700 Subject: [PATCH 27/38] IDEA doesn't optimize imports well for Scala, fixed missing ones --- .../mahout/cf/CooccurrenceAnalysis.scala | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index 32dddcce71..b01332c069 100644 --- a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -19,12 +19,14 @@ package org.apache.mahout.cf import org.apache.mahout.math._ import scalabindings._ +import RLikeOps._ import drm._ import RLikeDrmOps._ import scala.collection.JavaConversions._ import org.apache.mahout.math.stats.LogLikelihood import collection._ import org.apache.mahout.common.RandomUtils +import org.apache.mahout.math.function.{VectorFunction, Functions} /** @@ -100,19 +102,6 @@ object CooccurrenceAnalysis extends Serializable { val k12 = numInteractionsWithA - numInteractionsWithAandB val k21 = numInteractionsWithB - numInteractionsWithAandB val k22 = numInteractions - numInteractionsWithA - numInteractionsWithB + numInteractionsWithAandB -/* - long k11 = (long) element.get(); - long k12 = (long) (rowSums.get(row.index()) - k11); - long k21 = (long) (colSums.get(element.index()) - k11); - long k22 = (long) (total - k11 - k12 - k21); - double score = LogLikelihood.rootLogLikelihoodRatio(k11, k12, k21, k22); -*/ -/* - val k11 = numInteractionsWithAandB - val k12 = numInteractionsWithA - k11 - val k21 = numInteractionsWithB - k11 - val k22 = numInteractions - k11 - k12 - k21 -*/ LogLikelihood.logLikelihoodRatio(k11, k12, k21, k22) @@ -145,9 +134,8 @@ object CooccurrenceAnalysis extends Serializable { val llr = logLikelihoodRatio(numInteractionsB(thingB).toLong, numInteractionsA(thingA).toLong, cooccurrences.toLong, numUsers) - //not sure why this is calculated but it matches code in the hadoop version + // matches hadoop code and maps values to range (0..1) val tLLR = 1.0 - (1.0 / (1.0 + llr)) - val candidate = thingA -> tLLR // Enqueue item with score, if belonging to the top-k From c1a2e09a56f5617e73c8f74b4481572c65d36580 Mon Sep 17 00:00:00 2001 From: Pat Ferrel Date: Thu, 3 Jul 2014 15:46:39 -0700 Subject: [PATCH 28/38] bumped the JVM memory in the spark pom to 1025m, which avoid a java heap used up error in the build --- spark/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/pom.xml b/spark/pom.xml index 602f693253..03ea2a0030 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -166,7 +166,7 @@ ${project.build.directory}/scalatest-reports . WDF TestSuite.txt - -Xmx512m + -Xmx1024m From 9fb2027ec05f8078df05ba4ee8582eed553d3425 Mon Sep 17 00:00:00 2001 From: pferrel Date: Fri, 4 Jul 2014 10:59:38 -0700 Subject: [PATCH 29/38] permanent fix for tests, now they do not require MAHOUT_HOME or SPARK_HOME --- .../mahout/drivers/ItemSimilarityDriver.scala | 12 ++++++--- .../apache/mahout/drivers/MahoutDriver.scala | 4 +-- .../drivers/ItemSimilarityDriverSuite.scala | 25 +++++++++++-------- 3 files changed, 26 insertions(+), 15 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 77005f1ecd..1d3239830b 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -167,6 +167,11 @@ object ItemSimilarityDriver extends MahoutDriver { note("\nDefault delimiters will produce output of the form: \"itemID1itemID2:value2,itemID10:value10...\"") + //Jar inclusion, this option can be set when executing the driver from compiled code + opt[Unit]("dontAddMahoutJars") action { (_, options) => + options.copy(dontAddMahoutJars = true) //set the value MahoutDriver so the context will be created correctly + }//Hidden option, used when executing tests or calling from other code where classes are all loaded explicitly + //Driver notes--driver specific note("\nNote: Only the Log Likelihood Ratio (LLR) is supported as a similarity measure.\n") @@ -196,14 +201,14 @@ object ItemSimilarityDriver extends MahoutDriver { } override def start(masterUrl: String = options.master, - appName: String = options.appName): + appName: String = options.appName, dontAddMahoutJars: Boolean = options.dontAddMahoutJars): Unit = { sparkConf.set("spark.kryo.referenceTracking", "false") .set("spark.kryoserializer.buffer.mb", "200") .set("spark.executor.memory", options.sparkExecutorMem) - super.start(masterUrl, appName) + super.start(masterUrl, appName, dontAddMahoutJars) val readSchema1 = new Schema("delim" -> options.inDelim, "filter" -> options.filter1, "rowIDPosition" -> options.rowIDPosition, @@ -309,6 +314,7 @@ object ItemSimilarityDriver extends MahoutDriver { inDelim: String = "[,\t ]", rowKeyDelim: String = "\t", columnIdStrengthDelim: String = ":", - tupleDelim: String = ",") + tupleDelim: String = ",", + dontAddMahoutJars: Boolean = false) } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala index afc7c1e997..0c579d487e 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/MahoutDriver.scala @@ -65,9 +65,9 @@ abstract class MahoutDriver { mc = mahoutSparkContext(masterUrl, appName, customJars, sparkConf) } - protected def start(masterUrl: String, appName: String) : Unit = { + protected def start(masterUrl: String, appName: String, dontAddMahoutJars: Boolean = false) : Unit = { val customJars = Traversable.empty[String] - mc = mahoutSparkContext(masterUrl, appName, customJars, sparkConf) + mc = mahoutSparkContext(masterUrl, appName, customJars, sparkConf, !dontAddMahoutJars) } /** Override (optionally) for special cleanup */ diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index b524e55613..9f928e26b3 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -88,7 +88,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc )) */ - ignore ("ItemSimilarityDriver, non-full-spec CSV"){ + test ("ItemSimilarityDriver, non-full-spec CSV"){ val InFile = TmpDir + "in-file.csv/" //using part files, not singel file val OutPath = TmpDir + "indicator-matrices/" @@ -131,7 +131,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--inDelim", ",", "--itemIDPosition", "2", "--rowIDPosition", "0", - "--filterPosition", "1")) + "--filterPosition", "1", + "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] @@ -142,7 +143,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc - ignore ("ItemSimilarityDriver TSV "){ + test ("ItemSimilarityDriver TSV "){ val InFile = TmpDir + "in-file.tsv/" val OutPath = TmpDir + "indicator-matrices/" @@ -185,7 +186,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--inDelim", "[,\t]", "--itemIDPosition", "2", "--rowIDPosition", "0", - "--filterPosition", "1")) + "--filterPosition", "1", + "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] @@ -195,7 +197,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } - ignore ("ItemSimilarityDriver log-ish files"){ + test ("ItemSimilarityDriver log-ish files"){ val InFile = TmpDir + "in-file.log/" val OutPath = TmpDir + "indicator-matrices/" @@ -238,7 +240,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--inDelim", "\t", "--itemIDPosition", "4", "--rowIDPosition", "1", - "--filterPosition", "2")) + "--filterPosition", "2", + "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] @@ -248,7 +251,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } - ignore ("ItemSimilarityDriver legacy supported file format"){ + test ("ItemSimilarityDriver legacy supported file format"){ val InDir = TmpDir + "in-dir/" val InFilename = "in-file.tsv" @@ -286,7 +289,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc ItemSimilarityDriver.main(Array( "--input", InPath, "--output", OutPath, - "--master", masterUrl)) + "--master", masterUrl, + "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] @@ -294,7 +298,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc } - ignore("ItemSimilarityDriver recursive file discovery using filename patterns"){ + test("ItemSimilarityDriver recursive file discovery using filename patterns"){ //directory structure using the following // tmp/data/m1.tsv // tmp/data/more-data/another-dir/m2.tsv @@ -362,7 +366,8 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--rowIDPosition", "0", "--filterPosition", "1", "--filenamePattern", "m..tsv", - "--recursive")) + "--recursive", + "--dontAddMahoutJars")) beforeEach()// restart the test context to read the output of the driver val indicatorLines = mahoutCtx.textFile(OutPath + "/indicator-matrix/").collect.toSet[String] From a14ef4682fac6dbba749f1da7becfd3b21d06c88 Mon Sep 17 00:00:00 2001 From: pferrel Date: Fri, 4 Jul 2014 11:09:04 -0700 Subject: [PATCH 30/38] make the test --dontAddMahoutJars option hidden --- .../scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 1d3239830b..f78c59048d 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -168,7 +168,7 @@ object ItemSimilarityDriver extends MahoutDriver { note("\nDefault delimiters will produce output of the form: \"itemID1itemID2:value2,itemID10:value10...\"") //Jar inclusion, this option can be set when executing the driver from compiled code - opt[Unit]("dontAddMahoutJars") action { (_, options) => + opt[Unit]("dontAddMahoutJars") hidden() action { (_, options) => options.copy(dontAddMahoutJars = true) //set the value MahoutDriver so the context will be created correctly }//Hidden option, used when executing tests or calling from other code where classes are all loaded explicitly From aa77891a5a4706a9cd68e99769134508fc4251a0 Mon Sep 17 00:00:00 2001 From: pferrel Date: Mon, 7 Jul 2014 10:44:10 -0700 Subject: [PATCH 31/38] adding a two input option --- .../mahout/drivers/ItemSimilarityDriver.scala | 26 +++++++-- .../drivers/ItemSimilarityDriverSuite.scala | 56 +++++++++++++++++++ 2 files changed, 76 insertions(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index f78c59048d..bbd9ae810f 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -61,12 +61,16 @@ object ItemSimilarityDriver extends MahoutDriver { options.copy(input = x) } text ("Input path, may be a filename, directory name, or comma delimited list of HDFS supported URIs (required)") + opt[String]("input2") abbr ("i2") action { (x, options) => + options.copy(input2 = x) + } text ("Secondary input path for cross-similarity calculation, same restrictions as \"--input\" (optional). Default: empty. Note that the same input schema is applied to both \"--input\" and \"--input2\" files") + opt[String]('o', "output") required() action { (x, options) => if (x.endsWith("/")) // todo: check to see if HDFS allows MS-Windows backslashes locally? options.copy(output = x) else options.copy(output = x + "/") - } text ("Path for output, any local or HDFS supported URI (required).") + } text ("Path for output, any local or HDFS supported URI (required)") //Algorithm control options--driver specific note("\nAlgorithm control options:") @@ -113,7 +117,7 @@ object ItemSimilarityDriver extends MahoutDriver { opt[String]("filter2") abbr ("f2") action { (x, options) => options.copy(filter2 = x) - } text ("String (or regex) whose presence indicates a datum for the secondary item set (optional). If not present no secondary dataset is collected.") + } text ("String (or regex) whose presence indicates a datum for the secondary item set (optional). If not present no secondary dataset is collected") opt[Int]("rowIDPosition") abbr ("rc") action { (x, options) => options.copy(rowIDPosition = x) @@ -238,6 +242,7 @@ object ItemSimilarityDriver extends MahoutDriver { private def readIndexedDatasets: Array[IndexedDataset] = { val inFiles = FileSysUtils(options.input, options.filenamePattern, options.recursive).uris + val inFiles2 = if (options.input2 == null || options.input2.isEmpty) "" else FileSysUtils(options.input2, options.filenamePattern, options.recursive).uris if (inFiles.isEmpty) { Array() @@ -245,11 +250,19 @@ object ItemSimilarityDriver extends MahoutDriver { val selfSimilarityDataset = IndexedDataset(reader1.readFrom(inFiles)) - if (options.filterPosition != -1 && options.filter2 != null) { - // todo: needs to support more than one cross-similarity indicator - val crossSimilarityDataset1 = IndexedDataset(reader2.readFrom(inFiles)) - Array(selfSimilarityDataset, crossSimilarityDataset1) + if (!inFiles2.isEmpty) { + + // get cross-cooccurrence interactions from separate files + Array(selfSimilarityDataset, IndexedDataset(reader2.readFrom(inFiles2))) + + } else if (options.filterPosition != -1 && options.filter2 != null) { + + // get cross-cooccurrences interactions by filtering a single set of files + Array(selfSimilarityDataset, IndexedDataset(reader2.readFrom(inFiles))) + } else { + + // only return self-similarity A'A Array(selfSimilarityDataset) } @@ -301,6 +314,7 @@ object ItemSimilarityDriver extends MahoutDriver { randomSeed: Int = System.currentTimeMillis().toInt, recursive: Boolean = false, input: String = null, + input2: String = null, output: String = null, filenamePattern: String = "^part-.*", maxSimilaritiesPerItem: Int = 100, diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index 9f928e26b3..23eb85281a 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -376,6 +376,62 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc assert (crossIndicatorLines == CrossSimilarityTSV) } + test ("ItemSimilarityDriver, two input paths"){ + + val InFile1 = TmpDir + "in-file1.csv/" //using part files, not single file + val InFile2 = TmpDir + "in-file2.csv/" //using part files, not single file + val OutPath = TmpDir + "indicator-matrices/" + + val lines = Array( + "u1,purchase,iphone", + "u1,purchase,ipad", + "u2,purchase,nexus", + "u2,purchase,galaxy", + "u3,purchase,surface", + "u4,purchase,iphone", + "u4,purchase,galaxy", + "u1,view,iphone", + "u1,view,ipad", + "u1,view,nexus", + "u1,view,galaxy", + "u2,view,iphone", + "u2,view,ipad", + "u2,view,nexus", + "u2,view,galaxy", + "u3,view,surface", + "u3,view,nexus", + "u4,view,iphone", + "u4,view,ipad", + "u4,view,galaxy") + + // this will create multiple part-xxxxx files in the InFile dir but other tests will + // take account of one actual file + val linesRdd1 = mahoutCtx.parallelize(lines).saveAsTextFile(InFile1) + val linesRdd2 = mahoutCtx.parallelize(lines).saveAsTextFile(InFile2) + + afterEach // clean up before running the driver, it should handle the Spark conf and context + + // local multi-threaded Spark with default HDFS + ItemSimilarityDriver.main(Array( + "--input", InFile1, + "--input2", InFile2, + "--output", OutPath, + "--master", masterUrl, + "--filter1", "purchase", + "--filter2", "view", + "--inDelim", ",", + "--itemIDPosition", "2", + "--rowIDPosition", "0", + "--filterPosition", "1", + "--dontAddMahoutJars")) + + beforeEach // restart the test context to read the output of the driver + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == SelfSimilairtyTSV) + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] + assert (crossIndicatorLines == CrossSimilarityTSV) + } + override def afterAll = { // remove TmpDir val fs = FileSystem.get(new Configuration()) From 34320ae3b9c5c84892f6a6dc25fa0f5263771ffe Mon Sep 17 00:00:00 2001 From: pferrel Date: Fri, 11 Jul 2014 09:53:31 -0700 Subject: [PATCH 32/38] Yikes, the cross-cooccurrence stuff is in the wrong order--a big change so commit before making it --- .../main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index 9217f1a570..72dcf4dd09 100644 --- a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -74,7 +74,7 @@ object CooccurrenceAnalysis extends Serializable { val bcastInteractionsPerThingB = drmBroadcast(drmB.numNonZeroElementsPerColumn) // Compute cross-co-occurrence matrix B'A - // pferrel: yikes, this is the wrong order + // pferrel: yikes, this is the wrong order, a big change! so you know who to blame val drmBtA = drmB.t %*% drmA val drmIndicatorsBtA = computeIndicators(drmBtA, numUsers, maxInterestingItemsPerThing, From 9a3b09900bae6355e7360770fb288be478220c9e Mon Sep 17 00:00:00 2001 From: pferrel Date: Sat, 12 Jul 2014 15:23:47 -0700 Subject: [PATCH 33/38] fixing cross-coocurrence tests --- .../mahout/cf/CooccurrenceAnalysis.scala | 8 +++---- .../mahout/drivers/ItemSimilarityDriver.scala | 2 +- .../drivers/ItemSimilarityDriverSuite.scala | 23 +++++++++++-------- 3 files changed, 18 insertions(+), 15 deletions(-) diff --git a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala index 72dcf4dd09..e0dbdb2f5b 100644 --- a/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala +++ b/spark/src/main/scala/org/apache/mahout/cf/CooccurrenceAnalysis.scala @@ -75,12 +75,12 @@ object CooccurrenceAnalysis extends Serializable { // Compute cross-co-occurrence matrix B'A // pferrel: yikes, this is the wrong order, a big change! so you know who to blame - val drmBtA = drmB.t %*% drmA + val drmAtB = drmA.t %*% drmB - val drmIndicatorsBtA = computeIndicators(drmBtA, numUsers, maxInterestingItemsPerThing, - bcastInteractionsPerThingB, bcastInteractionsPerItemA) + val drmIndicatorsAtB = computeIndicators(drmAtB, numUsers, maxInterestingItemsPerThing, + bcastInteractionsPerItemA, bcastInteractionsPerThingB) - indicatorMatrices = indicatorMatrices :+ drmIndicatorsBtA + indicatorMatrices = indicatorMatrices :+ drmIndicatorsAtB drmB.uncache() } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 695a7144ff..3172203f15 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -347,7 +347,7 @@ object ItemSimilarityDriver extends MahoutDriver { // todo: needs to support more than one cross-similarity indicator if (indexedDatasets.length > 1) { - val crossIndicatorDataset = new IndexedDataset(indicatorMatrices(1), indexedDatasets(1).columnIDs, indexedDatasets(0).columnIDs) // cross similarity + val crossIndicatorDataset = new IndexedDataset(indicatorMatrices(1), indexedDatasets(0).columnIDs, indexedDatasets(1).columnIDs) // cross similarity writer.writeDRMTo(crossIndicatorDataset, options.output + "cross-indicator-matrix") } diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index 1220823888..45d5b5322b 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -44,14 +44,14 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc (0.0, 0.0, 0.0, 0.0, 0.8181382096075936)) */ - final val SelfSimilairtyTSV = Set( + final val SelfSimilairtyTSV = Array( "galaxy\tnexus:1.7260924347106847", "ipad\tiphone:1.7260924347106847", "nexus\tgalaxy:1.7260924347106847", "iphone\tipad:1.7260924347106847", "surface") - final val CrossSimilarityTSV = Set( + final val CrossSimilarityTSV = Array( "nexus\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,surface:0.6795961471815897,galaxy:1.7260924347106847", "ipad\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", "surface\tsurface:4.498681156950466", @@ -584,10 +584,11 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "u4,view,soap") val UnequalDimensionsCrossSimilarity = Set( - "galaxy\tsoap:0.6795961471815897,phones:1.7260924347106847,tablets:0.6795961471815897,null:0.6795961471815897,mobile_acc:1.7260924347106847", - "ipad\tmobile_acc:5.545177444479561,soap:1.7260924347106847", - "nexus\tmobile_acc:1.7260924347106847,phones:1.7260924347106847", - "iphone\tsoap:0.6795961471815897,phones:1.7260924347106847,tablets:0.6795961471815897,mobile_acc:1.7260924347106847") + "iphone\tmobile_acc:1.7260924347106847,soap:1.7260924347106847,phones:1.7260924347106847", + "surface\tmobil_acc:0.6795961471815897", + "nexus\tmobile_acc:0.6795961471815897,tablets:1.7260924347106847,phones:0.6795961471815897", + "galaxy\tsoap:1.7260924347106847,phones:1.7260924347106847,tablets:5.545177444479561,mobile_acc:1.7260924347106847", + "ipad\tmobile_acc:0.6795961471815897,phones:0.6795961471815897") // this will create multiple part-xxxxx files in the InFile dir but other tests will // take account of one actual file @@ -612,10 +613,12 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--writeAllDatasets")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == SelfSimilairtyTSV) - val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] - assert (crossIndicatorLines == UnequalDimensionsCrossSimilarity) + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect + val indicatorID = indicatorLines.hashCode + assert(indicatorID == SelfSimilairtyTSV.hashCode) + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect + val crossIndicatorID = crossIndicatorLines.hashCode + assert (crossIndicatorID == UnequalDimensionsCrossSimilarity.hashCode) } From 97ee4cc80e982b8dc139ad22f328e9f481e63171 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 17 Jul 2014 11:36:39 -0700 Subject: [PATCH 34/38] fixed CoocccurrenceAnalysis A'B instead of B'A, and fixed ItemSimilarityDriver to handle asymmetric A, and B input --- .../mahout/math/drm/CheckpointedDrm.scala | 6 + .../mahout/drivers/IndexedDataset.scala | 14 +- .../mahout/drivers/ItemSimilarityDriver.scala | 34 ++-- .../drivers/TextDelimitedReaderWriter.scala | 3 + .../drm/CheckpointedDrmSpark.scala | 18 +- .../mahout/cf/CooccurrenceAnalysisSuite.scala | 25 ++- .../drivers/ItemSimilarityDriverSuite.scala | 159 +++++++++++------- 7 files changed, 164 insertions(+), 95 deletions(-) diff --git a/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala b/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala index 0266944a80..f40caa38f8 100644 --- a/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala +++ b/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala @@ -33,4 +33,10 @@ trait CheckpointedDrm[K] extends DrmLike[K] { /** If this checkpoint is already declared cached, uncache. */ def uncache() + /** Adds n to row cardinality without modifying the underlying DRM data. + * does not perform an optimizer triggering action. + * @param n increase row cardinality by the number. + */ + def addToRowCardinality(n: Int): Unit + } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala index 2375315756..637791e9a2 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/IndexedDataset.scala @@ -45,9 +45,17 @@ case class IndexedDataset(var matrix: CheckpointedDrm[Int], var rowIDs: BiMap[St // we must allow the row dimension to be adjusted in the case where the data read in is incomplete and we // learn this afterwards - def nrow = matrix.nrow - def nrow_= (value: Long): Unit = matrix.asInstanceOf[CheckpointedDrmSpark[Int]].nrow = value - + /** + * Adds the equivalent of blank rows to the sparse CheckpointedDrm, which only changes the row cardinality value. + * No physical changes are made to the underlying drm. + * @param n number to increase row carnindality by + * @note should be done before any BLAS optimizer actions are performed on the matrix or you'll get unpredictable + * results. + */ + def addToRowCardinality(n: Int): Unit = { + assert(n > -1) + matrix.asInstanceOf[CheckpointedDrmSpark[Int]].addToRowCardinality(n) + } } /** diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index 3172203f15..a0ff6bb8df 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -283,39 +283,39 @@ object ItemSimilarityDriver extends MahoutDriver { } else { val selfSimilarityDataset = IndexedDataset(reader1.readTuplesFrom(inFiles)) - if (options.writeAllDatasets) writer.writeDRMTo(selfSimilarityDataset, options.output + "../input-datasets/matrix1") + val rowCardinalityA = selfSimilarityDataset.rowIDs.size() + if (options.writeAllDatasets) writer.writeDRMTo(selfSimilarityDataset, options.output + "../input-datasets/primary-interactions") - - // The case of B'A can be a bit sticky when the exact same row IDs don't exist for each dataset + // The case of A'B can be a bit sticky when the exact same row IDs don't exist for each dataset // Here we assume there is one row ID space for all interactions. To do this we pass in the // rowIDs created when reading selfSimilarityDataset and add to them if there are new ones in - // the second dataset. We will then apply the row dimension of the combined dataset to both DRMs - if (!inFiles2.isEmpty) { - + // the second dataset. We will then apply the row cardinality of the combined dataset to both DRMs + val indexedDatasetAtB = if (!inFiles2.isEmpty) { // get cross-cooccurrence interactions from separate files val crossSimilairtyDataset = IndexedDataset(reader2.readTuplesFrom(inFiles2, existingRowIDs = selfSimilarityDataset.rowIDs)) - selfSimilarityDataset.nrow = crossSimilairtyDataset.nrow // these may be larger than the nrow calculated earlier - if (options.writeAllDatasets) writer.writeDRMTo(crossSimilairtyDataset, options.output + "../input-datasets/matrix2") - - Array(selfSimilarityDataset, crossSimilairtyDataset) + crossSimilairtyDataset } else if (options.filterPosition != -1 && options.filter2 != null) { // get cross-cooccurrences interactions by filtering a single set of files val crossSimilairtyDataset = IndexedDataset(reader2.readTuplesFrom(inFiles, existingRowIDs = selfSimilarityDataset.rowIDs)) - if (options.writeAllDatasets) writer.writeDRMTo(crossSimilairtyDataset, options.output + "../input-datasets/matrix2") - - Array(selfSimilarityDataset, crossSimilairtyDataset) + crossSimilairtyDataset } else { - - // only return self-similarity A'A - Array(selfSimilarityDataset) - + null.asInstanceOf[IndexedDataset] } + if (indexedDatasetAtB != null.asInstanceOf[IndexedDataset]) { // did AtB calc + if (indexedDatasetAtB.rowIDs.size() > rowCardinalityA) { + val newRowIDsFound = indexedDatasetAtB.rowIDs.size() - rowCardinalityA + selfSimilarityDataset.addToRowCardinality(newRowIDsFound) // this forces cardinality to match + } + + if (options.writeAllDatasets) writer.writeDRMTo(indexedDatasetAtB, options.output + "../input-datasets/secondary-interactions") + Array(selfSimilarityDataset, indexedDatasetAtB) + } else Array(selfSimilarityDataset) } } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala index 46bbfa6569..01c398abbd 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/TextDelimitedReaderWriter.scala @@ -157,6 +157,9 @@ trait TDIndexedDatasetWriter extends Writer[IndexedDataset]{ matrix.rdd.map { case (rowID, itemVector) => + //often want the output rows sorted by element score + //if (sortVectors) { /* todo: sort the vectors by element score */ } + // each line is created of non-zero values with schema specified delimiters and original row and column ID tokens // first get the external rowID token var line: String = rowIDDictionary.inverse.get(rowID) + rowKeyDelim diff --git a/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala b/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala index 2115235f8f..ad4d782525 100644 --- a/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala +++ b/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala @@ -40,14 +40,26 @@ class CheckpointedDrmSpark[K: ClassTag]( override protected[mahout] val partitioningTag: Long = Random.nextLong() ) extends CheckpointedDrm[K] { - lazy val nrow = if (_nrow >= 0) _nrow else computeNRow + private var addedRowCardinality = 0 // increasing this has the effect of adding blank rows + lazy val nrow = if (_nrow >= 0) _nrow + addedRowCardinality else computeNRow + addedRowCardinality lazy val ncol = if (_ncol >= 0) _ncol else computeNCol - def nrow_= (value:Long): Unit = _nrow = value - private var cached: Boolean = false override protected[mahout] val context: DistributedContext = rdd.context + /** + * Adds the equivalent of blank rows to the sparse CheckpointedDrm, which only changes the + * [[org.apache.mahout.sparkbindings.drm +.CheckpointedDrmSpark#nrow]] value. + * No physical changes are made to the underlying rdd, now blank rows are added as would be done with rbind(blankRows) + * @param n number to increase row cardinality by + * @note should be done before any BLAS optimizer actions are performed on the matrix or you'll get unpredictable + * results. + */ + override def addToRowCardinality(n: Int): Unit = { + assert(n > -1) + addedRowCardinality = n + } /** * Action operator -- does not necessary means Spark action; but does mean running BLAS optimizer diff --git a/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala b/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala index b8961878b2..2433e1b02b 100644 --- a/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala @@ -19,6 +19,7 @@ package org.apache.mahout.cf import org.apache.mahout.math.drm._ import org.apache.mahout.math.scalabindings.{MatrixOps, _} +import org.apache.mahout.sparkbindings._ import org.apache.mahout.sparkbindings.test.MahoutLocalContext import org.apache.mahout.test.MahoutSuite import org.scalatest.FunSuite @@ -48,13 +49,19 @@ class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLoc (0.0, 0.0, 0.0, 0.0, 0.0)) // correct cross-cooccurrence with LLR - final val matrixLLRCoocBtAControl = dense( + final val m = dense( (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.0), (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.0), (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.6795961471815897), (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.0), (0.0, 0.0, 0.0, 0.0, 4.498681156950466)) + final val matrixLLRCoocBtAControl = dense( + (1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 0.0), + (0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.0), + (0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.0), + (1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 0.0), + (0.0, 0.0, 0.6795961471815897, 0.0, 4.498681156950466)) test("cooccurrence [A'A], [B'A] boolbean data using LLR") { @@ -164,19 +171,17 @@ class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLoc (1, 1, 0, 1)) val matrixLLRCoocBtANonSymmetric = dense( - (0.0, 0.0, 1.7260924347106847, 5.545177444479561, 0.0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.0), - (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 1.7260924347106847, 0.6795961471815897), - (1.7260924347106847, 0.0, 0.0, 1.7260924347106847, 0.0)) - + (0.0, 1.7260924347106847, 1.7260924347106847, 1.7260924347106847), + (0.0, 0.6795961471815897, 0.6795961471815897, 0.0), + (1.7260924347106847, 0.6795961471815897, 0.6795961471815897, 0.0), + (5.545177444479561, 1.7260924347106847, 1.7260924347106847, 1.7260924347106847), + (0.0, 0.0, 0.6795961471815897, 0.0)) val drmA = drmParallelize(m = a, numPartitions = 2) val drmB = drmParallelize(m = b, numPartitions = 2) //self similarity val drmCooc = CooccurrenceAnalysis.cooccurrences(drmARaw = drmA, drmBs = Array(drmB)) - //var cp = drmSelfCooc(0).checkpoint() - //cp.writeDRM("/tmp/cooc-spark/")//to get values written val matrixSelfCooc = drmCooc(0).checkpoint().collect val diffMatrix = matrixSelfCooc.minus(matrixLLRCoocAtAControl) var n = (new MatrixOps(m = diffMatrix)).norm @@ -186,6 +191,10 @@ class CooccurrenceAnalysisSuite extends FunSuite with MahoutSuite with MahoutLoc val matrixCrossCooc = drmCooc(1).checkpoint().collect val diff2Matrix = matrixCrossCooc.minus(matrixLLRCoocBtANonSymmetric) n = (new MatrixOps(m = diff2Matrix)).norm + + //cooccurrence without LLR is just a A'B + //val inCoreAtB = a.transpose().times(b) + //val bp = 0 } test("LLR calc") { diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index 45d5b5322b..a4ed37d7c8 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -37,26 +37,26 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc // correct cross-cooccurrence with LLR final val matrixLLRCoocBtAControl = dense( - (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), - (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), - (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.40461878191490940), - (0.6331745808516107, 0.4046187819149094, 0.4046187819149094, 0.6331745808516107, 0.0), - (0.0, 0.0, 0.0, 0.0, 0.8181382096075936)) + (1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 0.0), + (0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.0), + (0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.6795961471815897, 0.0), + (1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 1.7260924347106847, 0.0), + (0.0, 0.0, 0.6795961471815897, 0.0, 4.498681156950466)) */ - final val SelfSimilairtyTSV = Array( - "galaxy\tnexus:1.7260924347106847", - "ipad\tiphone:1.7260924347106847", - "nexus\tgalaxy:1.7260924347106847", - "iphone\tipad:1.7260924347106847", - "surface") + final val SelfSimilairtyTokens = tokenize(Iterable( + "galaxy\tnexus:1.7260924347106847", + "ipad\tiphone:1.7260924347106847", + "nexus\tgalaxy:1.7260924347106847", + "iphone\tipad:1.7260924347106847", + "surface")) - final val CrossSimilarityTSV = Array( - "nexus\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,surface:0.6795961471815897,galaxy:1.7260924347106847", - "ipad\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "surface\tsurface:4.498681156950466", - "iphone\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "galaxy\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847") + val CrossIndicatorTokens = Iterable("galaxy", "nexus:1.7260924347106847", "iphone:1.7260924347106847", + "ipad:1.7260924347106847", "galaxy:1.7260924347106847", "ipad", "nexus:0.6795961471815897", + "iphone:0.6795961471815897", "ipad:0.6795961471815897", "galaxy:0.6795961471815897", "nexus", + "nexus:0.6795961471815897", "iphone:0.6795961471815897", "ipad:0.6795961471815897", "galaxy:0.6795961471815897", + "iphone", "nexus:1.7260924347106847", "iphone:1.7260924347106847", "ipad:1.7260924347106847", + "galaxy:1.7260924347106847", "surface", "surface:4.498681156950466", "nexus:0.6795961471815897") final val TmpDir = "tmp/" // all IO going to whatever the default HDFS config is pointing to @@ -135,12 +135,16 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--rowIDPosition", "0", "--filterPosition", "1", "--dontAddMahoutJars")) + beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == SelfSimilairtyTSV) - val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] - assert (crossIndicatorLines == CrossSimilarityTSV) + + // todo: a better test would be to get sorted vectors and compare rows instead of tokens, this might miss + // some error cases + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens } @@ -192,10 +196,12 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == SelfSimilairtyTSV) - val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] - assert (crossIndicatorLines == CrossSimilarityTSV) + // todo: a better test would be to get sorted vectors and compare rows instead of tokens, this might miss + // some error cases + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens } @@ -246,10 +252,17 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == SelfSimilairtyTSV) +/* val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] + assert(indicatorLines == SelfSimilairtyTokens) val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] assert (crossIndicatorLines == CrossSimilarityTSV) +*/ + // todo: a better test would be to get sorted vectors and compare rows instead of tokens, this might miss + // some error cases + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens } @@ -270,12 +283,12 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "3,0,1", "3,3,1") - val Answer = Set( + val Answer = tokenize(Iterable( "0\t1:1.7260924347106847", "3\t2:1.7260924347106847", "1\t0:1.7260924347106847", "4", - "2\t3:1.7260924347106847") + "2\t3:1.7260924347106847")) // this creates one part-0000 file in the directory mahoutCtx.parallelize(lines).coalesce(1, shuffle=true).saveAsTextFile(InDir) @@ -295,8 +308,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == Answer) + // todo: a better test would be to get sorted vectors and compare rows instead of tokens, this might miss + // some error cases + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs Answer } @@ -317,12 +332,12 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "3,0,1", "3,3,1") - val Answer = Set( + val Answer = tokenize(Set( "0\t1", "3\t2", "1\t0", "4", - "2\t3") + "2\t3")) // this creates one part-0000 file in the directory mahoutCtx.parallelize(lines).coalesce(1, shuffle=true).saveAsTextFile(InDir) @@ -343,8 +358,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--omitStrength")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == Answer) + // todo: a better test would be to get sorted vectors and compare rows instead of tokens, this might miss + // some error cases + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs Answer } @@ -420,10 +437,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--dontAddMahoutJars")) beforeEach()// restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath + "/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == SelfSimilairtyTSV) - val crossIndicatorLines = mahoutCtx.textFile(OutPath + "/cross-indicator-matrix/").collect.toSet[String] - assert (crossIndicatorLines == CrossSimilarityTSV) + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens } test ("ItemSimilarityDriver, two input paths"){ @@ -476,10 +493,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == SelfSimilairtyTSV) - val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] - assert (crossIndicatorLines == CrossSimilarityTSV) + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens } test ("ItemSimilarityDriver, two inputs of different dimensions"){ @@ -512,18 +529,17 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "u4,view,ipad", "u4,view,galaxy") - val UnequalDimensionsSelfSimilarity = Set( + val UnequalDimensionsSelfSimilarity = tokenize(Iterable( "ipad\tiphone:1.7260924347106847", "iphone\tipad:1.7260924347106847", "nexus\tgalaxy:1.7260924347106847", - "galaxy\tnexus:1.7260924347106847") + "galaxy\tnexus:1.7260924347106847")) - val unequalDimensionsCrossSimilarity = Set( - "ipad\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "null", - "iphone\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "nexus\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847", - "galaxy\tnexus:0.6795961471815897,iphone:1.7260924347106847,ipad:0.6795961471815897,galaxy:1.7260924347106847") + val UnequalDimensionsCrossSimilarity = tokenize(Iterable( + "galaxy\tnexus:1.7260924347106847,iphone:1.7260924347106847,ipad:1.7260924347106847,galaxy:1.7260924347106847", + "ipad\tnexus:0.6795961471815897,iphone:0.6795961471815897,ipad:0.6795961471815897,galaxy:0.6795961471815897", + "nexus\tnexus:0.6795961471815897,iphone:0.6795961471815897,ipad:0.6795961471815897,galaxy:0.6795961471815897", + "iphone\tnexus:1.7260924347106847,iphone:1.7260924347106847,ipad:1.7260924347106847,galaxy:1.7260924347106847")) // this will create multiple part-xxxxx files in the InFile dir but other tests will // take account of one actual file @@ -547,10 +563,17 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--dontAddMahoutJars")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toSet[String] - assert(indicatorLines == UnequalDimensionsSelfSimilarity) - val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toSet[String] - assert (crossIndicatorLines == unequalDimensionsCrossSimilarity) +/* + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens + */ + + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs UnequalDimensionsSelfSimilarity + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines) should contain theSameElementsAs UnequalDimensionsCrossSimilarity } test("ItemSimilarityDriver cross similarity two separate items spaces"){ @@ -583,12 +606,12 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "u4,view,tablets", "u4,view,soap") - val UnequalDimensionsCrossSimilarity = Set( + val UnequalDimensionsCrossSimilarity = tokenize(Iterable( "iphone\tmobile_acc:1.7260924347106847,soap:1.7260924347106847,phones:1.7260924347106847", - "surface\tmobil_acc:0.6795961471815897", + "surface\tmobile_acc:0.6795961471815897", "nexus\tmobile_acc:0.6795961471815897,tablets:1.7260924347106847,phones:0.6795961471815897", "galaxy\tsoap:1.7260924347106847,phones:1.7260924347106847,tablets:5.545177444479561,mobile_acc:1.7260924347106847", - "ipad\tmobile_acc:0.6795961471815897,phones:0.6795961471815897") + "ipad\tmobile_acc:0.6795961471815897,phones:0.6795961471815897")) // this will create multiple part-xxxxx files in the InFile dir but other tests will // take account of one actual file @@ -613,13 +636,21 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with MahoutLoc "--writeAllDatasets")) beforeEach // restart the test context to read the output of the driver - val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect - val indicatorID = indicatorLines.hashCode - assert(indicatorID == SelfSimilairtyTSV.hashCode) - val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect - val crossIndicatorID = crossIndicatorLines.hashCode - assert (crossIndicatorID == UnequalDimensionsCrossSimilarity.hashCode) + val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable + tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens + val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable + tokenize(crossIndicatorLines) should contain theSameElementsAs UnequalDimensionsCrossSimilarity + } + // convert into an Iterable of tokens for 'should contain theSameElementsAs Iterable' + def tokenize(a: Iterable[String]): Iterable[String] = { + var r: Iterable[String] = Iterable() + a.foreach { l => + l.split("\t").foreach{ s => + r = r ++ s.split(",") + } + } + r.asInstanceOf[Iterable[String]] } override def afterAll = { From 0e7332deae11d8beaa0d508a274a848befb34501 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 17 Jul 2014 13:26:07 -0700 Subject: [PATCH 35/38] comment where bug was in cross-cooccurrence --- .../scala/org/apache/mahout/math/cf/CooccurrenceAnalysis.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/math-scala/src/main/scala/org/apache/mahout/math/cf/CooccurrenceAnalysis.scala b/math-scala/src/main/scala/org/apache/mahout/math/cf/CooccurrenceAnalysis.scala index c33859e35b..181b729d3c 100644 --- a/math-scala/src/main/scala/org/apache/mahout/math/cf/CooccurrenceAnalysis.scala +++ b/math-scala/src/main/scala/org/apache/mahout/math/cf/CooccurrenceAnalysis.scala @@ -75,6 +75,7 @@ object CooccurrenceAnalysis extends Serializable { // Compute cross-co-occurrence matrix B'A // pferrel: yikes, this is the wrong order, a big change! so you know who to blame + // used to be val drmBtA = drmB.t %*% drmA, which is the wrong order val drmAtB = drmA.t %*% drmB val drmIndicatorsAtB = computeIndicators(drmAtB, numUsers, maxInterestingItemsPerThing, From 7ca13614d31d60c35f26e2956273937f54677427 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 17 Jul 2014 13:29:49 -0700 Subject: [PATCH 36/38] not sure why the .DS_Stores keep creeping into the commits, argh --- spark/src/main/.DS_Store | Bin 6148 -> 0 bytes spark/src/main/scala/org/.DS_Store | Bin 6148 -> 0 bytes spark/src/main/scala/org/apache/.DS_Store | Bin 6148 -> 0 bytes 3 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 spark/src/main/.DS_Store delete mode 100644 spark/src/main/scala/org/.DS_Store delete mode 100644 spark/src/main/scala/org/apache/.DS_Store diff --git a/spark/src/main/.DS_Store b/spark/src/main/.DS_Store deleted file mode 100644 index 7ac63ad1120e5ae567aa3c8877093b929ddcb943..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6148 zcmeH~J&pn~427TBSqW(yO3E}GfEz@JJpmWk0m_jm5Pgom&yE{rsnKZlEIBWB;`#ZC z$rykg9@jmv0kEdKV&}umjPWP#xM0MD@p`{Z=i6=aG*-FS13Itqyq?Pv5fA|p5CIVo zfdvtWL!9UT*Mgo&k0JshunYqJeJFI-n%cU?r-MVZ0Mvo$FwUcwpcYS1YijGt49&88 zuq?G`Lp&el)ROyZYU|p|VcC3G-r2m1p;<476$Uh`AsR$L1ZD(2dc5)T|C|1*|DTm8 z6af+VX9R3Go{k5;RGzJWUeEJCne}<2Q)4@ar=I`@eiU!%VO%dhq1M#al^L3T1OkHw I5qK+sUmE!nYXATM diff --git a/spark/src/main/scala/org/.DS_Store b/spark/src/main/scala/org/.DS_Store deleted file mode 100644 index d6999d377b781f3027f076b1925a97c2fcd9c4c9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6148 zcmeHKI|@QE5ZqM}!N$@uSMUZw^aNhOFIWf)V*e`7<fKY%m813XGbB*6r9*;8fO-L>F+DnJEpT><+(6u4ncTmt>mfx%k<;0j?k%)OTY z77GAt;u44mOoIvxs%DF!K}Wn~UQJvAgD#rQhvvHE{`aI^s?T@@K$wp;3YVR^S&eHWjb{ From 7462be5fc7da259c100a057cfd84e5f9aff31002 Mon Sep 17 00:00:00 2001 From: pferrel Date: Thu, 17 Jul 2014 13:35:21 -0700 Subject: [PATCH 37/38] not sure why the .DS_Stores keep creeping into the commits, argh --- spark/src/main/scala/.DS_Store | Bin 6148 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 spark/src/main/scala/.DS_Store diff --git a/spark/src/main/scala/.DS_Store b/spark/src/main/scala/.DS_Store deleted file mode 100644 index e6dc460bb1c6d2efab7037b03d06d0556d348790..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6148 zcmeH~Jqp4=5QS%qgvIYIqf#4$mI7itHYo8^+Vg;}!TM!kPMk^St>SKu2 zy&WuhT}`%Nw2S8Op?PPuDF&v|E?SVlv^p5502LT1&_&+c`M-yMoBu~GOsN1B_%j7` zzB}x;c&R*FKVHx3$E@1A!9l+q;q4~?i5 Date: Sun, 20 Jul 2014 10:06:53 -0700 Subject: [PATCH 38/38] made addToRowCardinality return a new CheckpointedDrmSpark and so support immutability--still a question whether this is the right implementation of this. --- .../mahout/math/drm/CheckpointedDrm.scala | 2 +- .../mahout/drivers/ItemSimilarityDriver.scala | 2 +- .../drm/CheckpointedDrmSpark.scala | 7 +-- .../mahout/cf/CooccurrenceAnalysisSuite.scala | 18 ++++++ .../drivers/ItemSimilarityDriverSuite.scala | 60 +++++++++++++++++-- 5 files changed, 78 insertions(+), 11 deletions(-) diff --git a/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala b/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala index f40caa38f8..9bb15be0d8 100644 --- a/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala +++ b/math-scala/src/main/scala/org/apache/mahout/math/drm/CheckpointedDrm.scala @@ -37,6 +37,6 @@ trait CheckpointedDrm[K] extends DrmLike[K] { * does not perform an optimizer triggering action. * @param n increase row cardinality by the number. */ - def addToRowCardinality(n: Int): Unit + def addToRowCardinality(n: Int): CheckpointedDrm[K] } diff --git a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala index a0ff6bb8df..e01b931932 100644 --- a/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala +++ b/spark/src/main/scala/org/apache/mahout/drivers/ItemSimilarityDriver.scala @@ -17,7 +17,7 @@ package org.apache.mahout.drivers -import org.apache.mahout.cf.CooccurrenceAnalysis +import org.apache.mahout.math.cf.CooccurrenceAnalysis /** * Command line interface for [[org.apache.mahout.cf.CooccurrenceAnalysis.cooccurrences( )]]. diff --git a/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala b/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala index 297e592bde..232c2d1a00 100644 --- a/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala +++ b/spark/src/main/scala/org/apache/mahout/sparkbindings/drm/CheckpointedDrmSpark.scala @@ -40,8 +40,7 @@ class CheckpointedDrmSpark[K: ClassTag]( override protected[mahout] val partitioningTag: Long = Random.nextLong() ) extends CheckpointedDrm[K] { - private var addedRowCardinality = 0 // increasing this has the effect of adding blank rows - lazy val nrow = if (_nrow >= 0) _nrow + addedRowCardinality else computeNRow + addedRowCardinality + lazy val nrow = if (_nrow >= 0) _nrow else computeNRow lazy val ncol = if (_ncol >= 0) _ncol else computeNCol private var cached: Boolean = false @@ -56,9 +55,9 @@ class CheckpointedDrmSpark[K: ClassTag]( * @note should be done before any BLAS optimizer actions are performed on the matrix or you'll get unpredictable * results. */ - override def addToRowCardinality(n: Int): Unit = { + override def addToRowCardinality(n: Int): CheckpointedDrm[K] = { assert(n > -1) - addedRowCardinality = n + new CheckpointedDrmSpark[K](rdd, nrow + n, ncol, _cacheStorageLevel ) } /** diff --git a/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala b/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala index ab829907f6..e448ad1b1a 100644 --- a/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/cf/CooccurrenceAnalysisSuite.scala @@ -17,12 +17,30 @@ package org.apache.mahout.cf +import org.apache.mahout.math.cf.CooccurrenceAnalysis import org.apache.mahout.math.drm._ import org.apache.mahout.math.scalabindings.{MatrixOps, _} import org.apache.mahout.sparkbindings.test.DistributedSparkSuite import org.apache.mahout.test.MahoutSuite import org.scalatest.FunSuite + + + + +import org.apache.mahout.math._ +import scalabindings._ +import RLikeOps._ +import drm._ +import RLikeDrmOps._ +import scala.collection.JavaConversions._ +import org.apache.mahout.math.stats.LogLikelihood +import collection._ +import org.apache.mahout.common.RandomUtils +import org.apache.mahout.math.function.{VectorFunction, Functions} + + + /* values A = 1 1 0 0 0 diff --git a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala index ed78d3f722..26890fc422 100644 --- a/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala +++ b/spark/src/test/scala/org/apache/mahout/drivers/ItemSimilarityDriverSuite.scala @@ -19,6 +19,8 @@ package org.apache.mahout.drivers import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.mahout.math.drm._ +import org.apache.mahout.math.scalabindings._ import org.scalatest.FunSuite import org.apache.mahout.sparkbindings._ import org.apache.mahout.sparkbindings.test.DistributedSparkSuite @@ -536,10 +538,10 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with Distribut "galaxy\tnexus:1.7260924347106847")) val UnequalDimensionsCrossSimilarity = tokenize(Iterable( - "galaxy\tnexus:1.7260924347106847,iphone:1.7260924347106847,ipad:1.7260924347106847,galaxy:1.7260924347106847", - "ipad\tnexus:0.6795961471815897,iphone:0.6795961471815897,ipad:0.6795961471815897,galaxy:0.6795961471815897", - "nexus\tnexus:0.6795961471815897,iphone:0.6795961471815897,ipad:0.6795961471815897,galaxy:0.6795961471815897", - "iphone\tnexus:1.7260924347106847,iphone:1.7260924347106847,ipad:1.7260924347106847,galaxy:1.7260924347106847")) + "galaxy\tnexus:1.7260924347106847,iphone:1.7260924347106847,ipad:1.7260924347106847,galaxy:1.7260924347106847", + "ipad\tnexus:0.6795961471815897,iphone:0.6795961471815897,ipad:0.6795961471815897,galaxy:0.6795961471815897", + "nexus\tnexus:0.6795961471815897,iphone:0.6795961471815897,ipad:0.6795961471815897,galaxy:0.6795961471815897", + "iphone\tnexus:1.7260924347106847,iphone:1.7260924347106847,ipad:1.7260924347106847,galaxy:1.7260924347106847")) // this will create multiple part-xxxxx files in the InFile dir but other tests will // take account of one actual file @@ -568,7 +570,7 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with Distribut tokenize(indicatorLines) should contain theSameElementsAs SelfSimilairtyTokens val crossIndicatorLines = mahoutCtx.textFile(OutPath+"/cross-indicator-matrix/").collect.toIterable tokenize(crossIndicatorLines).toIterable should contain theSameElementsAs CrossIndicatorTokens - */ +*/ val indicatorLines = mahoutCtx.textFile(OutPath+"/indicator-matrix/").collect.toIterable tokenize(indicatorLines) should contain theSameElementsAs UnequalDimensionsSelfSimilarity @@ -661,4 +663,52 @@ class ItemSimilarityDriverSuite extends FunSuite with MahoutSuite with Distribut super.afterAll } + test("plus one"){ + val a = dense( + (1, 1), + (0, 0)) + + val drmA1 = drmParallelize(m = a, numPartitions = 2) + + // modified to return a new CheckpointedDrm so maintains immutability but still only increases the row cardinality + // by returning new CheckpointedDrmSpark[K](rdd, nrow + n, ncol, _cacheStorageLevel ) Hack for now. + val drmABigger1 = drmA1.addToRowCardinality(1) + + val drmABiggerPlusOne1 = drmABigger1.plus(1.0) // drmABigger has no row 2 in the rdd but an empty row 1 + // drmABiggerPlusOne1 is a dense matrix + println(drmABiggerPlusOne1) + + val drmA2 = drmParallelize(m = a, numPartitions = 2) + val drmABigger2 = drmA2.addToRowCardinality(1) + val drmABiggerPlusOne2 = drmABigger2 + 1.0 + drmABiggerPlusOne2.writeDRM("tmp/plus-one/drma-bigger-plus-one-ops/") + + + val bp = 0 + } + + test("multiply after adding rows"){ + val a = dense( + (1, 1), + (1, 1), + (1, 1)) + + val b = dense( + (1, 1)) + + val drmA = drmParallelize(m = a, numPartitions = 2) + val drmB = drmParallelize(m = b, numPartitions = 2) + + // modified to return a new CheckpointedDrm so maintains immutability but still only increases the row cardinality + // by returning new CheckpointedDrmSpark[K](rdd, nrow + n, ncol, _cacheStorageLevel ) Hack for now. + val drmBBigger = drmB.addToRowCardinality(2) + + val AtBBigger = drmA.t %*% drmBBigger + AtBBigger. .writeDRM("tmp/plus-one/drma-bigger-plus-one-ops/") + val inCoreAtBBigger = AtBBigger. + + val bp = 0 + } + + }