From ef18a4cead6913d7719488a7df13fac1a87b1b3c Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sat, 27 Feb 2016 14:26:10 +0800 Subject: [PATCH 1/6] create new pr --- .../spark/mllib/fpm/AssociationRules.scala | 249 ++++++++++++++++-- .../mllib/fpm/AssociationRulesSuite.scala | 97 +++++++ 2 files changed, 328 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 9a63cc29dacb5..394149d49fac0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.fpm import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.annotation.{Experimental, Since} @@ -26,24 +27,26 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.AssociationRules.Rule import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset import org.apache.spark.rdd.RDD +import org.apache.spark.SparkException /** * :: Experimental :: * * Generates association rules from a [[RDD[FreqItemset[Item]]]. This method only generates - * association rules which have a single item as the consequent. + * association rules whose consequent's length are no greater than maxConsequent. * */ @Since("1.5.0") @Experimental class AssociationRules private[fpm] ( - private var minConfidence: Double) extends Logging with Serializable { + private var minConfidence: Double, + private var maxConsequent: Int = 1) extends Logging with Serializable { /** * Constructs a default instance with default parameters {minConfidence = 0.8}. */ @Since("1.5.0") - def this() = this(0.8) + def this() = this(0.8, 1) /** * Sets the minimal confidence (default: `0.8`). @@ -56,31 +59,56 @@ class AssociationRules private[fpm] ( this } + /** + * Sets the maximum size of consequents used by Apriori Algorithm (default: `1`). + */ + @Since("2.0.0") + def setMaxConsequent(maxConsequent: Int): this.type = { + this.maxConsequent = maxConsequent + this + } + /** * Computes the association rules with confidence above [[minConfidence]]. + * * @param freqItemsets frequent itemset model obtained from [[FPGrowth]] - * @return a [[Set[Rule[Item]]] containing the association rules. + * @return a [[Set[Rule[Item]]] containing the assocation rules. * */ @Since("1.5.0") def run[Item: ClassTag](freqItemsets: RDD[FreqItemset[Item]]): RDD[Rule[Item]] = { - // For candidate rule X => Y, generate (X, (Y, freq(X union Y))) - val candidates = freqItemsets.flatMap { itemset => - val items = itemset.items - items.flatMap { item => - items.partition(_ == item) match { - case (consequent, antecedent) if !antecedent.isEmpty => - Some((antecedent.toSeq, (consequent.toSeq, itemset.freq))) - case _ => None + val sc = freqItemsets.sparkContext + val freqItems = freqItemsets.filter(_.items.length == 1).flatMap(_.items).collect() + val itemToRank = freqItems.zipWithIndex.toMap + val freqItemIndices = freqItemsets.map { + itemset => + val indices = itemset.items.flatMap(itemToRank.get).sorted.toSeq + (indices, itemset.freq) + } + + // Generate the initial candidates for 1-consequent rules. + val initCandidates = freqItemIndices.flatMap { + case (indices, freq) => + indices.flatMap { + index => + indices.partition(_ == index) match { + case (consequent, antecendent) if antecendent.nonEmpty => + Some((antecendent, (consequent, freq))) + case _ => None + } } - } } - // Join to get (X, ((Y, freq(X union Y)), freq(X))), generate rules, and filter by confidence - candidates.join(freqItemsets.map(x => (x.items.toSeq, x.freq))) - .map { case (antecendent, ((consequent, freqUnion), freqAntecedent)) => - new Rule(antecendent.toArray, consequent.toArray, freqUnion, freqAntecedent) - }.filter(_.confidence >= minConfidence) + // The initial empty rule set. + val initRules = sc.emptyRDD[(Seq[Int], Seq[Int], Long, Long)] + + val rules = genRules(freqItemIndices, initCandidates, 1, initRules) + rules.map { + case (antecendent, consequent, freqUnion, freqAntecedent) => + new Rule(antecendent.map(i => freqItems(i)).toArray, + consequent.map(i => freqItems(i)).toArray, + freqUnion, freqAntecedent) + } } /** Java-friendly version of [[run]]. */ @@ -89,6 +117,191 @@ class AssociationRules private[fpm] ( val tag = fakeClassTag[Item] run(freqItemsets.rdd)(tag) } + + /** + * Run the Apriori's Rule-Generation algorithm recursively. + * + * @param freqItemIndices Frequent items indices. + * @param candidates Candidates for rules with corresponding consequent-length. + * @param lenConsequent Consequent-length. + * @param rules Already generated rules. + * @return The generated rules. + * + */ + @Since("2.0.0") + private def genRules(freqItemIndices: RDD[(Seq[Int], Long)], + candidates: RDD[(Seq[Int], (Seq[Int], Long))], + lenConsequent: Int, + rules: RDD[(Seq[Int], Seq[Int], Long, Long)] + ): RDD[(Seq[Int], Seq[Int], Long, Long)] = { + + val numCandidates = candidates.count() + log.info(s"Candidates for $lenConsequent-consequent rules : $numCandidates") + if (numCandidates == 0 || lenConsequent > maxConsequent) { + rules + } else { + val sc = freqItemIndices.sparkContext + // Filter the candidates to get rules. + val newRules = candidates.join(freqItemIndices).flatMap { + case (antecendent, ((consequent, freqUnion), freqAntecedent)) + if freqUnion >= minConfidence * freqAntecedent => + Some(antecendent, consequent, freqUnion, freqAntecedent) + case _ => None + }.cache() + + val numNewRules = newRules.count() + log.info(s"Generated $lenConsequent-consequent rules : $numNewRules") + + if (numNewRules == 0) { + // No new rules generated. + rules + } else if (lenConsequent == maxConsequent) { + // Meet the consequent-length condition. + sc.union(rules, newRules) + } else { + // Generate the candidates for the next iteration. + val newCandidates = genCandidates(newRules, lenConsequent + 1) + // Generate the rules for the next iteration. + genRules(freqItemIndices, newCandidates, lenConsequent + 1, sc.union(rules, newRules)) + } + } + } + + /** + * Generate the candidate for the assigned consequent-length. + * + * @param newRules Rules whose consequent-length equals lenConsequent - 1. + * @param lenConsequent Consequent-length. + * @return The generated candidate. + * + */ + @Since("2.0.0") + private def genCandidates(newRules: RDD[(Seq[Int], Seq[Int], Long, Long)], + lenConsequent: Int): RDD[(Seq[Int], (Seq[Int], Long))] = { + // A n-consequent rule has n (n-1)-consequent parent rules, and there are n*(n-1)/2 + // pairs among them. If a n-consequent rule meet the confidence condition, all the + // n (n-1)-consequent parent rules should also meet the condition. That's to say, + // there are n*(n-1)/2 (n-1)-consequent rule pairs which can be merged into + // the n-consequent rule. + val numParis = lenConsequent * (lenConsequent - 1) / 2 + newRules.filter { + // rules whose antecendent length equals to 1 can not be used to generate new rules + case (antecendent, consequent, freqUnion, freqAntecedent) => + antecendent.size > 1 + }.map { + case (antecendent, consequent, freqUnion, freqAntecedent) => + val union = seqAdd(antecendent, consequent) + ((union, freqUnion), consequent) + }.groupByKey().filter { + case ((union, freqUnion), consequents) => + // To generate a n-consequent candidate, there should be at least n + // (n-1)-consequent parent rules. + consequents.size >= lenConsequent + }.flatMap { + case ((union, freqUnion), consequents) => + val array = consequents.toArray + val newConsequentCount = collection.mutable.Map[Seq[Int], Int]() + for (i <- 0 until array.length; j <- i + 1 until array.length) { + val newConsequent = seqAdd(array(i), array(j)) + if (newConsequent.length == lenConsequent) { + val cnt = newConsequentCount.getOrElse(newConsequent, 0) + newConsequentCount.update(newConsequent, cnt + 1) + } + } + newConsequentCount.filter { + // Candidates pruning. + case (newConsequents, cnt) => + cnt == numParis + }.keys.map { + newConsequent => + val newAntecendent = seqMinus(union, newConsequent) + (newAntecendent, (newConsequent, freqUnion)) + } + } + } + + /** + * Computes the union seq of two sorted seq. + * @param s1 ordered Seq1 + * @param s2 ordered Seq2 + * @return an ordered union Seq of s1 and s2. + * + */ + @Since("2.0.0") + private def seqAdd(s1: Seq[Int], s2: Seq[Int]): Seq[Int] = { + var i1 = 0 + var i2 = 0 + + val res = ArrayBuffer[Int]() + + while (i1 < s1.length && i2 < s2.length) { + val e1 = s1(i1) + val e2 = s2(i2) + + if (e1 == e2) { + res.append(e1) + i1 += 1 + i2 += 1 + } else if (e1 > e2) { + res.append(e2) + i2 += 1 + } else { + res.append(e1) + i1 += 1 + } + } + + if (i1 < s1.length) { + for (i <- i1 until s1.length) + res.append(s1(i)) + } else if (i2 < s2.length) { + for (i <- i2 until s2.length) + res.append(s2(i)) + } + + res + } + + /** + * Computes the complementary seq of two sorted seq. + * @param s1 ordered Seq1 + * @param s2 ordered Seq2, must be a sub-sequence of s1 + * @return an ordered Seq, which equals to s1 -- s2. + * + */ + @Since("2.0.0") + private def seqMinus(s1: Seq[Int], s2: Seq[Int]): Seq[Int] = { + var i1 = 0 + var i2 = 0 + + val res = ArrayBuffer[Int]() + + while (i1 < s1.length && i2 < s2.length) { + val e1 = s1(i1) + val e2 = s2(i2) + + if (e1 == e2) { + i1 += 1 + i2 += 1 + } else if (e1 < e2) { + res.append(e1) + i1 += 1 + } else { + throw new SparkException(s"AssociationRules.seqMinus :" + + s" ${s1.mkString(",")} is not a superset of ${s2.mkString(",")}") + } + } + + if (i1 < s1.length) { + for (i <- i1 until s1.length) + res.append(s1(i)) + } else if (i2 < s2.length) { + throw new SparkException(s"AssociationRules.seqMinus :" + + s" ${s1.mkString(",")} is not a superset of ${s2.mkString(",")}") + } + + res + } } @Since("1.5.0") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala index dcb1f398b04b8..998efeb22b0e5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala @@ -86,5 +86,102 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { assert(results2.size === 30) assert(results2.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) } + + test("association rules with defined consequent-length") { + val freqItemsets = sc.parallelize(Seq( + (Set("12"), 3L), (Set("2"), 5L), (Set("7"), 4L), (Set("11"), 3L), (Set("6"), 3L), + (Set("1"), 3L), + (Set("7", "2"), 3L), (Set("11", "6"), 3L), (Set("11", "7"), 3L), (Set("12", "7"), 3L), + (Set("6", "7"), 3L), (Set("6", "2"), 3L), (Set("11", "2"), 3L), + (Set("6", "7", "2"), 3L), (Set("11", "7", "2"), 3L), (Set("11", "6", "2"), 3L), + (Set("11", "6", "7"), 3L), + (Set("11", "6", "7", "2"), 3L) + ).map { + case (items, freq) => new FPGrowth.FreqItemset(items.toArray, freq) + }) + + val ar = new AssociationRules() + + val results1 = ar + .setMinConfidence(0.9) + .setMaxConsequent(Int.MaxValue) + .run(freqItemsets) + .collect() + + /* Verify results using the CMD with SPMF library (http://www.philippe-fournier-viger.com/spmf/) + Note: Arules package in R do not support rules with consequent-length larger than 1. + Note: Because SPMF's implementation of Association Rules only support integer as item, the above dataset + was encoded into integers. + echo '1 2 3 4 5\n2 6 7 8 9 10 11 12\n12 7 13 14 1\n7 2 6 15 11 12 16 17\n2\n7 2 6 1 16 11 5' > dataset + java -jar spmf.jar run FPGrowth_association_rules dataset rules1 50% 90% + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n]++} + END{for(i in s) print i,s[i]}' rules1 + > 1 23 + 2 12 + 3 2 + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n][$NF]++} + END{for(i in s)for(j in s[i]) print i,j,s[i][j]}' rules2 + > 1 1.0 23 + 2 1.0 12 + 3 1.0 2 + */ + assert(results1.size === 37) + assert(results1.filter(_.consequent.size == 1).size == 23) + assert(results1.filter(_.consequent.size == 2).size == 12) + assert(results1.filter(_.consequent.size == 3).size == 2) + assert(results1.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 37) + + val results2 = ar + .setMinConfidence(0.5) + .setMaxConsequent(Int.MaxValue) + .run(freqItemsets) + .collect() + + /* Verify results using the CMD with SPMF library (http://www.philippe-fournier-viger.com/spmf/) + Note: Arules package in R do not support rules with consequent-length larger than 1. + Note: Because SPMF's implementation of Association Rules only support integer as item, the above dataset + was encoded into integers. + echo '1 2 3 4 5\n2 6 7 8 9 10 11 12\n12 7 13 14 1\n7 2 6 15 11 12 16 17\n2\n7 2 6 1 16 11 5' > dataset + java -jar spmf.jar run FPGrowth_association_rules dataset rules2 50% 50% + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n]++} + END{for(i in s) print i,s[i]}' rules2 + > 1 30 + 2 18 + 3 4 + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n][$NF]++} + END{for(i in s)for(j in s[i]) print i,j,s[i][j]}' rules2 + > 1 0.6 3 + 1 0.75 4 + 1 1.0 23 + 2 0.6 3 + 2 0.75 3 + 2 1.0 12 + 3 0.6 1 + 3 0.75 1 + 3 1.0 2 + */ + assert(results2.size === 52) + assert(results2.filter(_.consequent.size == 1).size == 30) + assert(results2.filter(_.consequent.size == 2).size == 18) + assert(results2.filter(_.consequent.size == 3).size == 4) + assert(results2.count(rule => rule.consequent.size == 1 && + math.abs(rule.confidence - 0.6D) < 1e-6) == 3) + assert(results2.count(rule => rule.consequent.size == 1 && + math.abs(rule.confidence - 0.75D) < 1e-6) == 4) + assert(results2.count(rule => rule.consequent.size == 1 && + math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(results2.count(rule => rule.consequent.size == 2 && + math.abs(rule.confidence - 0.6D) < 1e-6) == 3) + assert(results2.count(rule => rule.consequent.size == 2 && + math.abs(rule.confidence - 0.75D) < 1e-6) == 3) + assert(results2.count(rule => rule.consequent.size == 2 && + math.abs(rule.confidence - 1.0D) < 1e-6) == 12) + assert(results2.count(rule => rule.consequent.size == 3 && + math.abs(rule.confidence - 0.6D) < 1e-6) == 1) + assert(results2.count(rule => rule.consequent.size == 3 && + math.abs(rule.confidence - 0.75D) < 1e-6) == 1) + assert(results2.count(rule => rule.consequent.size == 3 && + math.abs(rule.confidence - 1.0D) < 1e-6) == 2) + } } From 011be845c2522cc140e9217be790a18836562489 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sat, 27 Feb 2016 14:28:58 +0800 Subject: [PATCH 2/6] remove empty lines --- .../main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 394149d49fac0..82c6024629c9c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -70,7 +70,6 @@ class AssociationRules private[fpm] ( /** * Computes the association rules with confidence above [[minConfidence]]. - * * @param freqItemsets frequent itemset model obtained from [[FPGrowth]] * @return a [[Set[Rule[Item]]] containing the assocation rules. * From a50448a3b83f66f2cf9ffce3545bfd3a021718a3 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sat, 27 Feb 2016 16:49:00 +0800 Subject: [PATCH 3/6] format --- .../scala/org/apache/spark/mllib/fpm/AssociationRules.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 82c6024629c9c..bd7f81af63fcc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -39,8 +39,8 @@ import org.apache.spark.SparkException @Since("1.5.0") @Experimental class AssociationRules private[fpm] ( - private var minConfidence: Double, - private var maxConsequent: Int = 1) extends Logging with Serializable { + private var minConfidence: Double, + private var maxConsequent: Int = 1) extends Logging with Serializable { /** * Constructs a default instance with default parameters {minConfidence = 0.8}. From 7d204a99c0799f5a54146ebe31f11c89dc67d7f2 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sat, 27 Feb 2016 18:23:58 +0800 Subject: [PATCH 4/6] format test code --- .../mllib/fpm/AssociationRulesSuite.scala | 44 +++++++++++++------ 1 file changed, 30 insertions(+), 14 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala index 998efeb22b0e5..55a530b3f4860 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala @@ -110,17 +110,29 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { /* Verify results using the CMD with SPMF library (http://www.philippe-fournier-viger.com/spmf/) Note: Arules package in R do not support rules with consequent-length larger than 1. - Note: Because SPMF's implementation of Association Rules only support integer as item, the above dataset - was encoded into integers. - echo '1 2 3 4 5\n2 6 7 8 9 10 11 12\n12 7 13 14 1\n7 2 6 15 11 12 16 17\n2\n7 2 6 1 16 11 5' > dataset + Note: Because SPMF's implementation of Association Rules only support integer as item, + the above dataset was encoded into integers. + + Edit a file named 'dataset' with following lines: + 1 2 3 4 5 + 2 6 7 8 9 10 11 12 + 12 7 13 14 1 + 7 2 6 15 11 12 16 17 + 2 + 7 2 6 1 16 11 5 + java -jar spmf.jar run FPGrowth_association_rules dataset rules1 50% 90% - awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n]++} - END{for(i in s) print i,s[i]}' rules1 + + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} + {n=split($2,a," "); s[n]++} + END{for(i in s) print i,s[i]}' rules1 > 1 23 2 12 3 2 - awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n][$NF]++} - END{for(i in s)for(j in s[i]) print i,j,s[i][j]}' rules2 + + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} + {n=split($2,a," "); s[n][$NF]++} + END{for(i in s)for(j in s[i]) print i,j,s[i][j]}' rules1 > 1 1.0 23 2 1.0 12 3 1.0 2 @@ -139,17 +151,21 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { /* Verify results using the CMD with SPMF library (http://www.philippe-fournier-viger.com/spmf/) Note: Arules package in R do not support rules with consequent-length larger than 1. - Note: Because SPMF's implementation of Association Rules only support integer as item, the above dataset - was encoded into integers. - echo '1 2 3 4 5\n2 6 7 8 9 10 11 12\n12 7 13 14 1\n7 2 6 15 11 12 16 17\n2\n7 2 6 1 16 11 5' > dataset + Note: Because SPMF's implementation of Association Rules only support integer as item, + the above dataset was encoded into integers. + java -jar spmf.jar run FPGrowth_association_rules dataset rules2 50% 50% - awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n]++} - END{for(i in s) print i,s[i]}' rules2 + + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} + {n=split($2,a," "); s[n]++} + END{for(i in s) print i,s[i]}' rules2 > 1 30 2 18 3 4 - awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} {n=split($2,a," "); s[n][$NF]++} - END{for(i in s)for(j in s[i]) print i,j,s[i][j]}' rules2 + + awk 'BEGIN{FS="[=#:>]+"; PROCINFO["sorted_in"]="@ind_num_asc"} + {n=split($2,a," "); s[n][$NF]++} + END{for(i in s)for(j in s[i]) print i,j,s[i][j]}' rules2 > 1 0.6 3 1 0.75 4 1 1.0 23 From 794dd3cc19a3ebb55f7121d4e0bd6ecce8285706 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sun, 10 Apr 2016 09:49:28 +0800 Subject: [PATCH 5/6] fix confict --- .../scala/org/apache/spark/mllib/fpm/AssociationRules.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index bd7f81af63fcc..5a550b93c45ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -64,6 +64,8 @@ class AssociationRules private[fpm] ( */ @Since("2.0.0") def setMaxConsequent(maxConsequent: Int): this.type = { + require(maxConsequent > 0, + s"Maximum consequent must be positive but got ${maxConsequent}") this.maxConsequent = maxConsequent this } From 6868c1f5da856e622ffd330f25344c88a16a8841 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Sun, 10 Apr 2016 09:56:48 +0800 Subject: [PATCH 6/6] fix api --- .../org/apache/spark/mllib/fpm/AssociationRules.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 5a550b93c45ce..dc31a39ab8f7b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -39,14 +39,15 @@ import org.apache.spark.SparkException @Since("1.5.0") @Experimental class AssociationRules private[fpm] ( - private var minConfidence: Double, - private var maxConsequent: Int = 1) extends Logging with Serializable { + private var minConfidence: Double) extends Logging with Serializable { + + private var maxConsequent: Int = 1 /** * Constructs a default instance with default parameters {minConfidence = 0.8}. */ @Since("1.5.0") - def this() = this(0.8, 1) + def this() = this(0.8) /** * Sets the minimal confidence (default: `0.8`).