From da2cba7e063745aacef74ff555e7bd7c55a24f56 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 19 Oct 2014 02:19:27 -0700 Subject: [PATCH 01/10] adding apriori algorithm for frequent item set mining in Spark --- .../org/apache/spark/mllib/fim/Apriori.scala | 153 ++++++++++++++++++ .../apache/spark/mllib/fim/AprioriSuite.scala | 137 ++++++++++++++++ 2 files changed, 290 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala new file mode 100644 index 0000000000000..e956f6338ae63 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fim + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.{Logging, SparkContext} + +/** + * This object implements Apriori algorithm using Spark to find frequent item set in the given data set. + */ +object Apriori extends Logging with Serializable { + + /** + * Generate the first round FIS(frequent item set) from input data set. Returns single distinct item that + * appear greater than minCount times. + * + * @param dataSet input data set + * @param minCount the minimum appearance time that computed from minimum degree of support + * @return FIS + */ + private def genFirstRoundFIS(dataSet: RDD[Set[String]], + minCount: Double): RDD[(Set[String], Int)] = { + dataSet.flatMap(line => line) + .map(v => (v, 1)) + .reduceByKey(_ + _) + .filter(_._2 >= minCount) + .map(x => (Set(x._1), x._2)) + } + + /** + * Scan the input data set and filter out the eligible FIS + * @param candidate candidate FIS + * @param minCount the minimum appearance time that computed from minimum degree of support + * @return FIS + */ + private def scanAndFilter(dataSet: RDD[Set[String]], + candidate: RDD[Set[String]], + minCount: Double, + sc: SparkContext): RDD[(Set[String], Int)] = { + + dataSet.cartesian(candidate).map(x => + if (x._2.subsetOf(x._1)) { + (x._2, 1) + } else { + (x._2, 0) + }).reduceByKey(_+_).filter(x => x._2 >= minCount) + } + + /** + * Generate the next round of FIS candidate using this round FIS + * @param FISk + * @param k + * @return candidate FIS + */ + private def generateCombination(FISk: RDD[Set[String]], + k: Int): RDD[Set[String]] = { + FISk.cartesian(FISk) + .map(x => x._1 ++ x._2) + .filter(x => x.size == k) + .distinct() + } + + /** + * Function of apriori algorithm implementation. + * + * @param input Input data set to find frequent item set + * @param minSupport The minimum degree of support + * @param sc SparkContext to use + * @return frequent item sets in a array + */ + def apriori(input: RDD[Array[String]], + minSupport: Double, + sc: SparkContext): Array[(Set[String], Int)] = { + + /* + * This apriori implementation uses cartesian of two RDD, input data set and candidate + * FIS (frequent item set). + * The resulting FIS are computed in two steps: + * The first step, find eligible distinct item in data set. + * The second step, loop in k round, in each round generate candidate FIS and filter out eligible FIS + */ + + // calculate minimum appearance count for minimum degree of support + val dataSetLen: Long = input.count() + val minCount = minSupport * dataSetLen + + // This algorithm finds frequent item set, so convert each element of RDD to set to + // eliminate duplicated item if any + val dataSet = input.map(_.toSet) + + // FIS is the result to return + val FIS = collection.mutable.ArrayBuffer[RDD[(Set[String], Int)]]() + val FIS1: RDD[(Set[String], Int)] = genFirstRoundFIS(dataSet, minCount) + if (FIS1.count() < 0) { + return Array[(Set[String], Int)]() + } + + FIS += FIS1 + + // FIS for round k + var FISk = FIS1 + // round counter + var k = 2 + + while (FIS(k - 2).count() > 1) { + + // generate candidate FIS + val candidate: RDD[Set[String]] = generateCombination(FIS(k - 2).map(x => x._1), k) + + // filter out eligible FIS + FISk = scanAndFilter(dataSet, candidate, minCount, sc) + + // add it to the result and go to next round + FIS += FISk + k = k + 1 + } + + // convert all FIS to array before returning them + val retArr = collection.mutable.ArrayBuffer[(Set[String], Int)]() + for (l <- FIS) { + retArr.appendAll(l.collect()) + } + retArr.toArray + } + + private def printFISk(FIS: RDD[(Set[String], Int)], k: Int) { + print("FIS" + (k - 2) + " size " + FIS.count() + " value: ") + FIS.collect().foreach(x => print("(" + x._1 + ", " + x._2 + ") ")) + println() + } + + private def printCk(Ck: RDD[Set[String]], k: Int) { + print("C" + (k - 2) + " size "+ Ck.count() + " value: ") + Ck.collect().foreach(print) + println() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala new file mode 100644 index 0000000000000..1398d904f4501 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala @@ -0,0 +1,137 @@ +/* + * Licensed until the Apache Software Foundation (ASF) under one or more + * contribuuntilr license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file until 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 until in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fim + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.scalatest.FunSuite + +/** + * scala test unit + * using Practical Machine Learning Book data set to test apriori algorithm + */ +class AprioriSuite extends FunSuite with LocalSparkContext { + + test("test FIM with Apriori dataset 1") + { + + // input data set + val input = Array[String]( + "1 3 4", + "2 3 5", + "1 2 3 5", + "2 5") + + // correct FIS answers + val answer1 = Array((Set("4")), (Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("4", "1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("2", "1")), (Set("5", "1")), (Set("4", "3")), (Set("5", "2", "3")), (Set("3", "1", "5")), (Set("3", "1", "2")), (Set("4", "1", "3")), (Set("5", "2", "1")), (Set("5", "2", "3", "1"))) + val answer2 = Array((Set("4")), (Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("4", "1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("2", "1")), (Set("5", "1")), (Set("4", "3")), (Set("5", "2", "3")), (Set("3", "1", "5")), (Set("3", "1", "2")), (Set("4", "1", "3")), (Set("5", "2", "1")), (Set("5", "2", "3", "1"))) + val answer3 = Array((Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("5", "2", "3"))) + val answer4 = Array((Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("5", "2", "3"))) + val answer5 = Array((Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("5", "2", "3"))) + val answer6 = Array((Set("5")), (Set("2")), (Set("3")), (Set("5", "2"))) + val answer7 = Array((Set("5")), (Set("2")), (Set("3")), (Set("5", "2"))) + val answer8 = Array() + val answer9 = Array() + + val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= Apriori.apriori + + val dataSet = sc.parallelize(input) + val rdd = dataSet.map(line => line.split(" ")) + + val result9 = target(rdd, 0.9, sc) + assert(result9.length == answer9.length) + + val result8 = target(rdd, 0.8, sc) + assert(result8.length == answer8.length) + + val result7 = target(rdd, 0.7, sc) + assert(result7.length == answer7.length) + for (i <- 0 until result7.length){ + assert(answer7(i).equals(result7(i)._1)) + } + + val result6 = target(rdd, 0.6, sc) + assert(result6.length == answer6.length) + for (i <- 0 until result6.length) + assert(answer6(i).equals(result6(i)._1)) + + val result5 = target(rdd, 0.5, sc) + assert(result5.length == answer5.length) + for (i <- 0 until result5.length) + assert(answer5(i).equals(result5(i)._1)) + + val result4 = target(rdd, 0.4, sc) + assert(result4.length == answer4.length) + for (i <- 0 until result4.length) + assert(answer4(i).equals(result4(i)._1)) + + val result3 = target(rdd, 0.3, sc) + assert(result3.length == answer3.length) + for (i <- 0 until result3.length) + assert(answer3(i).equals(result3(i)._1)) + + val result2 = target(rdd, 0.2, sc) + assert(result2.length == answer2.length) + for (i <- 0 until result2.length) + assert(answer2(i).equals(result2(i)._1)) + + val result1 = target(rdd, 0.1, sc) + assert(result1.length == answer1.length) + for (i <- 0 until result1.length) + assert(answer1(i).equals(result1(i)._1)) + } + + test("test FIM with Apriori dataset 2") + { + + // input data set + val input = Array[String]( + "r z h j p", + "z y x w v u t s", + "z", + "r x n o s", + "y r x z q t p", + "y z x e q s t m") + + val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= Apriori.apriori + + val dataSet = sc.parallelize(input) + val rdd = dataSet.map(line => line.split(" ")) + + assert(target(rdd,0.9,sc).length == 0) + + assert(target(rdd,0.8,sc).length == 1) + + assert(target(rdd,0.7,sc).length == 1) + + assert(target(rdd,0.6,sc).length == 2) + + assert(target(rdd,0.5,sc).length == 18) + + assert(target(rdd,0.4,sc).length == 18) + + assert(target(rdd,0.3,sc).length == 54) + + assert(target(rdd,0.2,sc).length == 54) + + assert(target(rdd,0.1,sc).length == 625) + + } + +} From 889b33fdfabcc222c82e3bce619aeb6c7031fc58 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 19 Oct 2014 02:31:04 -0700 Subject: [PATCH 02/10] modify per scalastyle check --- .../org/apache/spark/mllib/fim/Apriori.scala | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala index e956f6338ae63..c9d73b8c59604 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala @@ -22,20 +22,21 @@ import org.apache.spark.SparkContext._ import org.apache.spark.{Logging, SparkContext} /** - * This object implements Apriori algorithm using Spark to find frequent item set in the given data set. + * This object implements Apriori algorithm using Spark to find frequent item set in + * the given data set. */ object Apriori extends Logging with Serializable { /** - * Generate the first round FIS(frequent item set) from input data set. Returns single distinct item that - * appear greater than minCount times. - * + * Generate the first round FIS(frequent item set) from input data set. Returns single + * distinct item that appear greater than minCount times. + * * @param dataSet input data set * @param minCount the minimum appearance time that computed from minimum degree of support * @return FIS */ private def genFirstRoundFIS(dataSet: RDD[Set[String]], - minCount: Double): RDD[(Set[String], Int)] = { + minCount: Double): RDD[(Set[String], Int)] = { dataSet.flatMap(line => line) .map(v => (v, 1)) .reduceByKey(_ + _) @@ -50,16 +51,16 @@ object Apriori extends Logging with Serializable { * @return FIS */ private def scanAndFilter(dataSet: RDD[Set[String]], - candidate: RDD[Set[String]], - minCount: Double, - sc: SparkContext): RDD[(Set[String], Int)] = { + candidate: RDD[Set[String]], + minCount: Double, + sc: SparkContext): RDD[(Set[String], Int)] = { dataSet.cartesian(candidate).map(x => if (x._2.subsetOf(x._1)) { (x._2, 1) } else { (x._2, 0) - }).reduceByKey(_+_).filter(x => x._2 >= minCount) + }).reduceByKey(_ + _).filter(x => x._2 >= minCount) } /** @@ -69,7 +70,7 @@ object Apriori extends Logging with Serializable { * @return candidate FIS */ private def generateCombination(FISk: RDD[Set[String]], - k: Int): RDD[Set[String]] = { + k: Int): RDD[Set[String]] = { FISk.cartesian(FISk) .map(x => x._1 ++ x._2) .filter(x => x.size == k) @@ -85,15 +86,16 @@ object Apriori extends Logging with Serializable { * @return frequent item sets in a array */ def apriori(input: RDD[Array[String]], - minSupport: Double, - sc: SparkContext): Array[(Set[String], Int)] = { + minSupport: Double, + sc: SparkContext): Array[(Set[String], Int)] = { /* * This apriori implementation uses cartesian of two RDD, input data set and candidate * FIS (frequent item set). * The resulting FIS are computed in two steps: * The first step, find eligible distinct item in data set. - * The second step, loop in k round, in each round generate candidate FIS and filter out eligible FIS + * The second step, loop in k round, in each round generate candidate FIS and filter + * out eligible FIS */ // calculate minimum appearance count for minimum degree of support @@ -144,9 +146,9 @@ object Apriori extends Logging with Serializable { FIS.collect().foreach(x => print("(" + x._1 + ", " + x._2 + ") ")) println() } - + private def printCk(Ck: RDD[Set[String]], k: Int) { - print("C" + (k - 2) + " size "+ Ck.count() + " value: ") + print("C" + (k - 2) + " size " + Ck.count() + " value: ") Ck.collect().foreach(print) println() } From f68a0bdf46ad5e91295ae334dd643bc1ecdb12a0 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 27 Nov 2014 01:04:47 +0800 Subject: [PATCH 03/10] add 2 apriori implemenation and fp-growth implementation --- .../org/apache/spark/mllib/fim/Apriori.scala | 155 ----------- .../spark/mllib/fim/AprioriByBroadcast.scala | 244 ++++++++++++++++++ .../spark/mllib/fim/AprioriByCartesian.scala | 146 +++++++++++ .../org/apache/spark/mllib/fim/FPGrowth.scala | 238 +++++++++++++++++ .../apache/spark/mllib/fim/AprioriSuite.scala | 159 ++++++------ .../spark/mllib/fim/FPGrowthSuite.scala | 82 ++++++ 6 files changed, 795 insertions(+), 229 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala deleted file mode 100644 index c9d73b8c59604..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/Apriori.scala +++ /dev/null @@ -1,155 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.fim - -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ -import org.apache.spark.{Logging, SparkContext} - -/** - * This object implements Apriori algorithm using Spark to find frequent item set in - * the given data set. - */ -object Apriori extends Logging with Serializable { - - /** - * Generate the first round FIS(frequent item set) from input data set. Returns single - * distinct item that appear greater than minCount times. - * - * @param dataSet input data set - * @param minCount the minimum appearance time that computed from minimum degree of support - * @return FIS - */ - private def genFirstRoundFIS(dataSet: RDD[Set[String]], - minCount: Double): RDD[(Set[String], Int)] = { - dataSet.flatMap(line => line) - .map(v => (v, 1)) - .reduceByKey(_ + _) - .filter(_._2 >= minCount) - .map(x => (Set(x._1), x._2)) - } - - /** - * Scan the input data set and filter out the eligible FIS - * @param candidate candidate FIS - * @param minCount the minimum appearance time that computed from minimum degree of support - * @return FIS - */ - private def scanAndFilter(dataSet: RDD[Set[String]], - candidate: RDD[Set[String]], - minCount: Double, - sc: SparkContext): RDD[(Set[String], Int)] = { - - dataSet.cartesian(candidate).map(x => - if (x._2.subsetOf(x._1)) { - (x._2, 1) - } else { - (x._2, 0) - }).reduceByKey(_ + _).filter(x => x._2 >= minCount) - } - - /** - * Generate the next round of FIS candidate using this round FIS - * @param FISk - * @param k - * @return candidate FIS - */ - private def generateCombination(FISk: RDD[Set[String]], - k: Int): RDD[Set[String]] = { - FISk.cartesian(FISk) - .map(x => x._1 ++ x._2) - .filter(x => x.size == k) - .distinct() - } - - /** - * Function of apriori algorithm implementation. - * - * @param input Input data set to find frequent item set - * @param minSupport The minimum degree of support - * @param sc SparkContext to use - * @return frequent item sets in a array - */ - def apriori(input: RDD[Array[String]], - minSupport: Double, - sc: SparkContext): Array[(Set[String], Int)] = { - - /* - * This apriori implementation uses cartesian of two RDD, input data set and candidate - * FIS (frequent item set). - * The resulting FIS are computed in two steps: - * The first step, find eligible distinct item in data set. - * The second step, loop in k round, in each round generate candidate FIS and filter - * out eligible FIS - */ - - // calculate minimum appearance count for minimum degree of support - val dataSetLen: Long = input.count() - val minCount = minSupport * dataSetLen - - // This algorithm finds frequent item set, so convert each element of RDD to set to - // eliminate duplicated item if any - val dataSet = input.map(_.toSet) - - // FIS is the result to return - val FIS = collection.mutable.ArrayBuffer[RDD[(Set[String], Int)]]() - val FIS1: RDD[(Set[String], Int)] = genFirstRoundFIS(dataSet, minCount) - if (FIS1.count() < 0) { - return Array[(Set[String], Int)]() - } - - FIS += FIS1 - - // FIS for round k - var FISk = FIS1 - // round counter - var k = 2 - - while (FIS(k - 2).count() > 1) { - - // generate candidate FIS - val candidate: RDD[Set[String]] = generateCombination(FIS(k - 2).map(x => x._1), k) - - // filter out eligible FIS - FISk = scanAndFilter(dataSet, candidate, minCount, sc) - - // add it to the result and go to next round - FIS += FISk - k = k + 1 - } - - // convert all FIS to array before returning them - val retArr = collection.mutable.ArrayBuffer[(Set[String], Int)]() - for (l <- FIS) { - retArr.appendAll(l.collect()) - } - retArr.toArray - } - - private def printFISk(FIS: RDD[(Set[String], Int)], k: Int) { - print("FIS" + (k - 2) + " size " + FIS.count() + " value: ") - FIS.collect().foreach(x => print("(" + x._1 + ", " + x._2 + ") ")) - println() - } - - private def printCk(Ck: RDD[Set[String]], k: Int) { - print("C" + (k - 2) + " size " + Ck.count() + " value: ") - Ck.collect().foreach(print) - println() - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala new file mode 100644 index 0000000000000..7dc7bea26d166 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.mllib.fim + +import org.apache.spark.SparkContext._ +import org.apache.spark.broadcast._ +import org.apache.spark.rdd.RDD +import org.apache.spark.{Logging, SparkContext} + +/** + * Calculate frequent item set using Apriori algorithm with minSupport. + * The apriori algorithm have two steps: + * step one is scan the data set to get L1 by minSuppprt + * step two is scan the data set multiple to get Lk + */ +object AprioriByBroadcast extends Logging with Serializable { + + /** + * Create C1 which contains all of single item in data Set. + * + * @param dataSet For mining frequent itemsets dataset + * @return single item in data Set + */ + def createC1(dataSet: RDD[Array[String]]): Array[Array[String]] = { + //get all distinct item in the RDD + val itemCollection = dataSet.flatMap(line => line).distinct().collect() + + //define new array which item is an array form + val itemArrCollection = collection.mutable.ArrayBuffer[Array[String]]() + + //change the itemsCollection into itemArrCollection + for (item <- itemCollection) { + itemArrCollection += Array[String](item) + } + + itemArrCollection.toArray + } + + + /** + * create Lk from Ck.Lk is generated by Ck when the frequent of Ck bigger than minCount + * @param dataSet For mining frequent itemsets dataset + * @param Ck Candidate set + * @param minCount The minimum degree of support + * @return Lk + */ + def scanD(dataSet: RDD[Array[String]], + Ck: Array[Array[String]], + minCount: Double, + sc: SparkContext): Array[(Array[String], Int)] = { + //broadcast Ck + val broadcastCk = sc.broadcast(Ck) + //val broadcastCkList: Array[Array[String]] = broadcastCk.value + + val Lk = dataSet.flatMap(line => containCk(line, broadcastCk)) + .filter(_.length > 0) + .map(v => (v, 1)) + .reduceByKey(_ + _) + .filter(_._2 >= (minCount)) + .map(v => (v._1.split(" "), v._2)) + .collect() + + Lk + } + + + /** + * containCk method. + * @param line dataset line + * @param broadcastCk L1 + * @return get Ck array + */ + def containCk(line: Array[String], + broadcastCk: Broadcast[Array[Array[String]]]): Array[String] = { + + // Ck broadcast value + val broadcastCkList: Array[Array[String]] = broadcastCk.value + // + var dbLineArrayBuffer = collection.mutable.ArrayBuffer[String]() + // the count number + var k: Int = 0 + + for (broadcastCk <- broadcastCkList) { + val bdArray: Array[String] = broadcastCk.sortWith((s, t) => s.compareTo(t) < 0).array + + if (bdArray.toSet subsetOf (line.toSet)) { + val bdString: String = bdArray.mkString(" ") + dbLineArrayBuffer ++= Array[(String)](bdString) + k = k + 1 + } + } + + if (k == 0) { + dbLineArrayBuffer ++= Array[(String)]("") + } + dbLineArrayBuffer.toArray.array + } + + /** + * create Ck by Lk + * @param Lk + * @param k + * @return Ck + */ + def aprioriGen(Lk: Array[(Array[String], Int)], + k: Int): Array[Array[String]] = { + + val LkLen = Lk.length + val CkBuffer = collection.mutable.ArrayBuffer[Array[String]]() + + //get Ck from Lk + for (i <- 0 to LkLen - 1) + for (j <- i + 1 to LkLen - 1) { + // get Lk:k-2 before k-2 item + val L1: Array[String] = + Lk(i)._1.take(k - 2).sortWith((s, t) => s.compareTo(t) < 0) + val L2: Array[String] = + Lk(j)._1.take(k - 2).sortWith((s, t) => s.compareTo(t) < 0) + + // merge set while the two set L1 and L2 equals + if (L1.mkString.equals(L2.mkString)) { + CkBuffer.append((Lk(i)._1.toSet ++ Lk(j)._1.toSet).toArray) + } + } + + if (CkBuffer.length > 0) { + CkBuffer.toArray.array + } else { + null + } + } + + /** + * create L1 + * @param dataSet For mining frequent item sets dataset + * @param minCount The minimum degree of support + * @return L1 + */ + def aprioriStepOne(dataSet: RDD[Array[String]], + minCount: Double): Array[(Array[String], Int)] = { + dataSet.flatMap(line => line) + .map(v => (v, 1)) + .reduceByKey(_ + _) + .filter(_._2 >= minCount) + .map(v => line2Array(v)) + .collect() + } + + /** + * change line type + * @param line line type (String,Int) + * @return line tpye (Array[String],Int) + */ + def line2Array(line: (String, Int)): (Array[String], Int) = { + (Array[String](line._1), line._2) + } + + /** + * apriori algorithm. + * Solving frequent item sets based on the data set and the minimum degree of support. + * The first phase, the scan time data sets, computing frequent item sets L1. + * The second stage, multiple scan data sets, computing frequent item sets Lk. + * @param dataSet For mining frequent item sets dataset + * @param minSupport The minimum degree of support + * @param sc + * @return frequent item sets + */ + def apriori(dataSet: RDD[Array[String]], + minSupport: Double, + sc: SparkContext): Array[(Set[String], Int)] = { + + //dataSet length + val dataSetLen: Long = dataSet.count() + //the count line for minSupport + val minCount = minSupport * dataSetLen + + //definite L collection that using save all of frequent item set + val L = collection.mutable.ArrayBuffer[Array[(Array[String], Int)]]() + val FIS = collection.mutable.ArrayBuffer[(Set[String], Int)]() + + //call aprioriStepOne method to get L1 + val L1: Array[(Array[String], Int)] = aprioriStepOne(dataSet, minCount) + logDebug("L1 length:" + L1.length) + logDebug("L1:" + L1) + + // L1 assignment to L + if (L1.length > 0) { + L += L1 + + for (arr <- L1) { + FIS.append((arr._1.toSet, arr._2)) + } + + // step counter + var k: Int = 2 + // do the loop while the k > 0 and L length > 1 + while ((k > 0) && L(k - 2).length > 1) { + + //call createCk method to get Ck + val Ck: Array[Array[String]] = aprioriGen(L(k - 2), k) + + if (Ck != null) { + //call createLk method to get Lk + val Lk: Array[(Array[String], Int)] = + scanD( + dataSet, + Ck, + minCount, + sc) + // Lk assignment to L + L += Lk + + for (arr <- Lk) { + FIS.append((arr._1.toSet, arr._2)) + } + + k = k + 1 + } + else { + k = -1 + } + } + FIS.toArray + } + else { + Array[(Set[String], Int)]() + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala new file mode 100644 index 0000000000000..bd256bac10c0f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fim + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.{Logging, SparkContext} + +/** + * Calculate frequent item set using Apriori algorithm with minSupport. + * The apriori algorithm have two steps: + * step one is scan the data set to get L1 by minSuppprt + * step two is scan the data set multiple to get Lk + */ +object AprioriByCartesian extends Logging with Serializable { + + /** + * create L1 + * @param dataSet For mining frequent item sets dataset + * @param minCount The minimum degree of support + * @return L1 + */ + def aprioriStepOne(dataSet: RDD[Set[String]], + minCount: Double): RDD[(Set[String], Int)] = { + dataSet.flatMap(line => line) + .map(v => (v, 1)) + .reduceByKey(_ + _) + .filter(_._2 >= minCount) + .map(x => (Set(x._1), x._2)) + } + + /** + * create Lk from Ck.Lk is generated by Ck when the frequent of Ck bigger than minSupport + * @param Ck Candidate set + * @param minCount The minimum degree of support + * @return Lk + */ + def scanD(dataSet: RDD[Set[String]], + Ck: RDD[Set[String]], + minCount: Double, + sc: SparkContext): RDD[(Set[String], Int)] = { + + dataSet.cartesian(Ck).map(x => + if (x._2.subsetOf(x._1)) { + (x._2, 1) + } else { + (x._2, 0) + }).reduceByKey(_ + _).filter(x => x._2 >= minCount) + } + + /** + * create Ck by Lk + * @param Lk + * @param k + * @return Ck + */ + def aprioriGen(Lk: RDD[Set[String]], + k: Int): RDD[Set[String]] = { + Lk.cartesian(Lk) + .map(x => x._1 ++ x._2) + .filter(x => x.size == k) + .distinct() + } + + /** + * apriori algorithm using cartesian of two RDD. + * Solving frequent item sets based on the data set and the minimum degree of support. + * The first phase, the scan time data sets, computing frequent item sets L1. + * The second stage, multiple scan data sets, computing frequent item sets Lk. + * @param input For mining frequent item sets dataset + * @param minSupport The minimum degree of support + * @param sc + * @return frequent item sets + */ + def apriori(input: RDD[Array[String]], + minSupport: Double, + sc: SparkContext): Array[(Set[String], Int)] = { + + //dataSet length + val dataSetLen: Long = input.count() + //the count line for minSupport + val minCount = minSupport * dataSetLen + // This algorithm finds frequent item set, so convert each element of RDD to set + val dataSet = input.map(_.toSet) + + //definite L collection that using save all of frequent item set + val L = collection.mutable.ArrayBuffer[RDD[(Set[String], Int)]]() + + val L1: RDD[(Set[String], Int)] = aprioriStepOne(dataSet, minCount) + if (L1.count() > 0) { + L += L1 + var Lk = L1 + + // step counter + var k = 2 + + while (L(k - 2).count() > 1) { + + // get candidate of frequent item set + val Ck: RDD[Set[String]] = aprioriGen(L(k - 2).map(x => x._1), k) + + // scan input data set to calculate degree of support for each candidate, + // and filter out the one not ineligible + Lk = scanD(dataSet, Ck, minCount, sc) + + k = k + 1 + L += Lk + } + //return all result in L + val retArr = collection.mutable.ArrayBuffer[(Set[String], Int)]() + for (l <- L) { + retArr.appendAll(l.collect()) + } + retArr.toArray + } else { + Array[(Set[String], Int)]() + } + } + + def printLk(Lk: RDD[(Set[String], Int)], k: Int) { + print("L" + (k - 2) + " size " + Lk.count() + " value: ") + Lk.collect().foreach(x => print("(" + x._1 + ", " + x._2 + ") ")) + println() + } + + def printCk(Ck: RDD[Set[String]], k: Int) { + print("C" + (k - 2) + " size " + Ck.count() + " value: ") + Ck.collect().foreach(print) + println() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala new file mode 100644 index 0000000000000..fc2db8e32073e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fim + +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.broadcast._ + +/** + * calculate frequent item set using FPGrowth algorithm with dada set and minSupport + * the FPGrowth algorithm have two step task + * step one is scaning data db to get L1 by minSuppprt + * step two is scan data db once to get Lk + */ +class FPGrowth extends Logging with Serializable { + + /** + * FPGrowth algorithm: + * step 1:calculate L1 by minSupport + * step 2: calculate Ln by FP-Tree + * @param sc sparkContext + * @param RDD For mining frequent item sets dataset + * @param minSuport The minimum degree of support + * @return frequent item sets + */ + def FPGrowth(RDD: RDD[Array[String]], + minSuport: Double, + sc: SparkContext): Array[(String, Int)] = { + val count = RDD.count() + logDebug("data set count:" + count) + val minCount = minSuport * count + logDebug("minSuppot count:" + minSuport) + + //one times scan data db to get L1 + val L1 = FPGStepOne(RDD, minCount) + logDebug("L1 length:" + L1.length) + logDebug("L1:" + L1) + + //two times scan data db to get Ln + val Ln = FPGStepTwo(sc, RDD, minCount, L1) + //add L1 and Ln to get fim + val fim = L1 ++ Ln + + return fim + + } + + /** + * Step 1: calculate L1 by min support + * @param RDD For mining frequent item sets dataset + * @param minCount The minimum degree of support + * @return L1 + */ + def FPGStepOne(RDD: RDD[Array[String]], + minCount: Double): Array[(String, Int)] = { + RDD.flatMap(v => v) + .map(v => (v, 1)) + .reduceByKey(_ + _) + .filter(_._2 >= minCount) + .collect() + .distinct + .sortWith(_._2 > _._2) + } + + /** + * step 2: using PFP-Tree to calculate the fim + * @param sc sparkContext + * @param RDD For mining frequent item sets dataset + * @param minCount The minimum degree of support + * @param L1 frenauent item set as length 1 + * @return Ln + */ + def FPGStepTwo(sc: SparkContext, + RDD: RDD[Array[String]], + minCount: Double, + L1: Array[(String, Int)]): Array[(String, Int)] = { + //broadcast L1 + val bdL1 = sc.broadcast(L1) + //val bdL1List = bdL1.value + + RDD.flatMap(line => L12LineMap(line, bdL1)) + .groupByKey() + .flatMap(line => FPTree(line, minCount)) + .collect() + + } + + /** + * create CFP-Tree + * give L1,example:a 2,b 4,c 3 and give line,example: a,b,c,d + * after calculate,the result is: + * a,(b,c) + * c,(b) + * note,the result have not b,() + * @param line dataset line + * @param bdL1 L1 + * @return CFP-Tree + */ + def L12LineMap(line: Array[String], + bdL1: Broadcast[Array[(String, Int)]]): Array[(String, Array[String])] = { + // broadcast value + val bdL1List = bdL1.value + // the result variable + var lineArrayBuffer = collection.mutable.ArrayBuffer[(String, Int)]() + + for (item <- line) { + + val opt = bdL1List.find(_._1.equals(item)) + + if (opt != None) { + lineArrayBuffer ++= opt + } + + } + + // sort array + val lineArray = lineArrayBuffer + .sortWith(_._1 > _._1) + .sortWith(_._2 > _._2) + .toArray + + + var arrArrayBuffer = collection.mutable.ArrayBuffer[(String, Array[String])]() + + /** + * give (a,4) (b 3),(c,3),after + * b,((a,4) + * c,((a,4) (b 3)) + */ + var arrBuffer = collection.mutable.ArrayBuffer[String]() + for (item <- lineArray) { + val arr = lineArray.take(lineArray.indexOf(item)) + + arrBuffer.clear() + + if (arr.length > 0) { + for (tempArr <- arr) { + //remain key + arrBuffer += tempArr._1 + } + arrArrayBuffer += ((item._1, arrBuffer.toArray)) + } + + } + + return arrArrayBuffer.toArray + + } + + /** + * genarate fim set by FPTree,everyone node have a CPFTree that can combination frenquent item + * @param line dataset line + * @param minCount The minimum degree of support + * @return fim + */ + def FPTree(line: (String, Iterable[Array[String]]), minCount: Double): Array[(String, Int)] = { + // frequently item + val key = line._1 + // the set of construction CPFTree + val value = line._2 + + val _lineBuffer = collection.mutable.ArrayBuffer[(String, Int)]() + val map = scala.collection.mutable.Map[String, Int]() + // tree step + var k = 1 + // loop the data set while k>0 + while (k > 0) { + map.clear() + + // loop data set + for (it <- value) { + if (it.length >= k) { + // from m get n combinations,using scala method + val lineCom = it.toList.combinations(k).toList + + // add key to combination + for (item <- lineCom) { + // sort array + val list2key: List[String] = (item :+ key) + .sortWith(_ > _) + + val s = list2key.mkString(" ") + + if (map.get(s) == None) { + map(s) = 1 + } + else { + map(s) = map.apply(s) + 1 + } + } + } + } + + var line: Array[(String, Int)] = null + + if (map.size != 0) { + // get fim set + val lineTemp = map.filter(_._2 >= minCount) + + if (lineTemp.size != 0) { + line = lineTemp.toArray.array + _lineBuffer ++= line + } + + } + + // reset k value + if ((line == null) || (line.length == 0)) { + k = 0 + } + else { + k = k + 1 + } + + } + + return _lineBuffer.toArray + + } + +} + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala index 1398d904f4501..e08e6b6ffb41a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala @@ -1,20 +1,19 @@ /* - * Licensed until the Apache Software Foundation (ASF) under one or more - * contribuuntilr license agreements. See the NOTICE file distributed with + * 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 until You under the Apache License, Version 2.0 + * 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 until in writing, software + * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.spark.mllib.fim import org.apache.spark.SparkContext @@ -24,94 +23,75 @@ import org.scalatest.FunSuite /** * scala test unit - * using Practical Machine Learning Book data set to test apriori algorithm + * using Practical Machine Learning Book data test the apriori algorithm result by minSupport from 0.9 to 0.1 */ class AprioriSuite extends FunSuite with LocalSparkContext { - test("test FIM with Apriori dataset 1") + test("test FIM with AprioriByBroadcast dataset 1") { + val arr = AprioriSuite.createFIMDataSet1() + val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByBroadcast.apriori - // input data set - val input = Array[String]( - "1 3 4", - "2 3 5", - "1 2 3 5", - "2 5") + val dataSet = sc.parallelize(arr) + val rdd = dataSet.map(line => line.split(" ")) - // correct FIS answers - val answer1 = Array((Set("4")), (Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("4", "1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("2", "1")), (Set("5", "1")), (Set("4", "3")), (Set("5", "2", "3")), (Set("3", "1", "5")), (Set("3", "1", "2")), (Set("4", "1", "3")), (Set("5", "2", "1")), (Set("5", "2", "3", "1"))) - val answer2 = Array((Set("4")), (Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("4", "1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("2", "1")), (Set("5", "1")), (Set("4", "3")), (Set("5", "2", "3")), (Set("3", "1", "5")), (Set("3", "1", "2")), (Set("4", "1", "3")), (Set("5", "2", "1")), (Set("5", "2", "3", "1"))) - val answer3 = Array((Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("5", "2", "3"))) - val answer4 = Array((Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("5", "2", "3"))) - val answer5 = Array((Set("5")), (Set("2")), (Set("3")), (Set("1")), (Set("5", "2")), (Set("3", "1")), (Set("5", "3")), (Set("2", "3")), (Set("5", "2", "3"))) - val answer6 = Array((Set("5")), (Set("2")), (Set("3")), (Set("5", "2"))) - val answer7 = Array((Set("5")), (Set("2")), (Set("3")), (Set("5", "2"))) - val answer8 = Array() - val answer9 = Array() + for (i <- 1 to 9){ + println(s"frequent item set with support ${i/10d}") + target(rdd, i/10d, sc).foreach(x => print("(" + x._1 + "), ")) + println() + } + } - val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= Apriori.apriori + test("test FIM with AprioriByBroadcast dataset 2") + { + val arr = AprioriSuite.createFIMDataSet2() + val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByBroadcast.apriori - val dataSet = sc.parallelize(input) + val dataSet = sc.parallelize(arr) val rdd = dataSet.map(line => line.split(" ")) - val result9 = target(rdd, 0.9, sc) - assert(result9.length == answer9.length) + assert(target(rdd,0.9,sc).length == 0) - val result8 = target(rdd, 0.8, sc) - assert(result8.length == answer8.length) + assert(target(rdd,0.8,sc).length == 1) - val result7 = target(rdd, 0.7, sc) - assert(result7.length == answer7.length) - for (i <- 0 until result7.length){ - assert(answer7(i).equals(result7(i)._1)) - } + assert(target(rdd,0.7,sc).length == 1) + + assert(target(rdd,0.6,sc).length == 2) + + assert(target(rdd,0.5,sc).length == 18) + + assert(target(rdd,0.4,sc).length == 18) + + assert(target(rdd,0.3,sc).length == 54) + + assert(target(rdd,0.2,sc).length == 54) + + assert(target(rdd,0.1,sc).length == 625) - val result6 = target(rdd, 0.6, sc) - assert(result6.length == answer6.length) - for (i <- 0 until result6.length) - assert(answer6(i).equals(result6(i)._1)) - - val result5 = target(rdd, 0.5, sc) - assert(result5.length == answer5.length) - for (i <- 0 until result5.length) - assert(answer5(i).equals(result5(i)._1)) - - val result4 = target(rdd, 0.4, sc) - assert(result4.length == answer4.length) - for (i <- 0 until result4.length) - assert(answer4(i).equals(result4(i)._1)) - - val result3 = target(rdd, 0.3, sc) - assert(result3.length == answer3.length) - for (i <- 0 until result3.length) - assert(answer3(i).equals(result3(i)._1)) - - val result2 = target(rdd, 0.2, sc) - assert(result2.length == answer2.length) - for (i <- 0 until result2.length) - assert(answer2(i).equals(result2(i)._1)) - - val result1 = target(rdd, 0.1, sc) - assert(result1.length == answer1.length) - for (i <- 0 until result1.length) - assert(answer1(i).equals(result1(i)._1)) } - test("test FIM with Apriori dataset 2") + test("test FIM with AprioriByCartesian dataset 1") { + val arr = AprioriSuite.createFIMDataSet1() + val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByCartesian.apriori - // input data set - val input = Array[String]( - "r z h j p", - "z y x w v u t s", - "z", - "r x n o s", - "y r x z q t p", - "y z x e q s t m") + val dataSet = sc.parallelize(arr) + val rdd = dataSet.map(line => line.split(" ")) + + for (i <- 1 to 9){ + println(s"frequent item set with support ${i/10d}") + target(rdd, i/10d, sc).foreach(x => print("(" + x._1 + "), ")) + println() + } - val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= Apriori.apriori + + } + test("test FIM with AprioriByCartesian dataset 2") + { + val arr = AprioriSuite.createFIMDataSet2() + val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByCartesian.apriori - val dataSet = sc.parallelize(input) + val dataSet = sc.parallelize(arr) val rdd = dataSet.map(line => line.split(" ")) assert(target(rdd,0.9,sc).length == 0) @@ -135,3 +115,34 @@ class AprioriSuite extends FunSuite with LocalSparkContext { } } + +/** + * create dataset + */ +object AprioriSuite +{ + /** + * create dataset using Practical Machine Learning Book data + * @return dataset + */ + def createFIMDataSet1():Array[String] = { + val arr = Array[String]( + "1 3 4", + "2 3 5", + "1 2 3 5", + "2 5") + arr + } + + def createFIMDataSet2():Array[String] = { + val arr = Array[String]( + "r z h j p", + "z y x w v u t s", + "z", + "r x n o s", + "y r x z q t p", + "y z x e q s t m") + arr + } +} + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala new file mode 100644 index 0000000000000..5156babcd1975 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.mllib.fim + +import org.scalatest.FunSuite +import org.apache.spark.mllib.util.LocalSparkContext + +/** + * scala test unit + * using Practical Machine Learning Book data test the FPGrowth algorithm result by minSupport from 0.9 to 0.1 + */ +class FPGrowthSuite extends FunSuite with LocalSparkContext { + + test("test FIM with FPGrowth") + { + val arr = AprioriSuite.createFIMDataSet2() + + assert(arr.length === 6) + val dataSet = sc.parallelize(arr) + assert(dataSet.count() == 6) + val rdd = dataSet.map(line => line.split(" ")) + assert(rdd.count() == 6) + + //check frenquent item set length + val fimWithFPGrowth = new FPGrowth() + println(fimWithFPGrowth.FPGrowth(rdd,0.1,sc).length) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.9,sc).length == 0) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.8,sc).length == 1) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.7,sc).length == 1) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.6,sc).length == 2) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.5,sc).length == 18) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.4,sc).length == 18) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.3,sc).length == 54) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.2,sc).length == 54) + + assert(fimWithFPGrowth.FPGrowth(rdd,0.1,sc).length == 625) + + } +} + +/** + * create dataset + */ +object FPGrowthSuite +{ + /** + * create dataset using Practical Machine Learning Book data + * @return dataset + */ + def createFIMDataSet():Array[String] = + { + val arr = Array[String]("r z h j p", + "z y x w v u t s", + "z", + "r x n o s", + "y r x z q t p", + "y z x e q s t m") + return arr + } +} From 7b77ad74a7de9af8e720f37430ce4b75651298be Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 27 Nov 2014 01:29:49 +0800 Subject: [PATCH 04/10] fix scalastyle check --- .../spark/mllib/fim/AprioriByBroadcast.scala | 24 +++++----- .../spark/mllib/fim/AprioriByCartesian.scala | 8 ++-- .../org/apache/spark/mllib/fim/FPGrowth.scala | 46 +++++++------------ 3 files changed, 32 insertions(+), 46 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala index 7dc7bea26d166..ca67c6069d0f3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala @@ -36,13 +36,13 @@ object AprioriByBroadcast extends Logging with Serializable { * @return single item in data Set */ def createC1(dataSet: RDD[Array[String]]): Array[Array[String]] = { - //get all distinct item in the RDD + // get all distinct item in the RDD val itemCollection = dataSet.flatMap(line => line).distinct().collect() - //define new array which item is an array form + // define new array which item is an array form val itemArrCollection = collection.mutable.ArrayBuffer[Array[String]]() - //change the itemsCollection into itemArrCollection + // change the itemsCollection into itemArrCollection for (item <- itemCollection) { itemArrCollection += Array[String](item) } @@ -62,10 +62,8 @@ object AprioriByBroadcast extends Logging with Serializable { Ck: Array[Array[String]], minCount: Double, sc: SparkContext): Array[(Array[String], Int)] = { - //broadcast Ck + // broadcast Ck val broadcastCk = sc.broadcast(Ck) - //val broadcastCkList: Array[Array[String]] = broadcastCk.value - val Lk = dataSet.flatMap(line => containCk(line, broadcastCk)) .filter(_.length > 0) .map(v => (v, 1)) @@ -122,7 +120,7 @@ object AprioriByBroadcast extends Logging with Serializable { val LkLen = Lk.length val CkBuffer = collection.mutable.ArrayBuffer[Array[String]]() - //get Ck from Lk + // get Ck from Lk for (i <- 0 to LkLen - 1) for (j <- i + 1 to LkLen - 1) { // get Lk:k-2 before k-2 item @@ -183,16 +181,16 @@ object AprioriByBroadcast extends Logging with Serializable { minSupport: Double, sc: SparkContext): Array[(Set[String], Int)] = { - //dataSet length + // dataSet length val dataSetLen: Long = dataSet.count() - //the count line for minSupport + // the count line for minSupport val minCount = minSupport * dataSetLen - //definite L collection that using save all of frequent item set + // definite L collection that using save all of frequent item set val L = collection.mutable.ArrayBuffer[Array[(Array[String], Int)]]() val FIS = collection.mutable.ArrayBuffer[(Set[String], Int)]() - //call aprioriStepOne method to get L1 + // call aprioriStepOne method to get L1 val L1: Array[(Array[String], Int)] = aprioriStepOne(dataSet, minCount) logDebug("L1 length:" + L1.length) logDebug("L1:" + L1) @@ -210,11 +208,11 @@ object AprioriByBroadcast extends Logging with Serializable { // do the loop while the k > 0 and L length > 1 while ((k > 0) && L(k - 2).length > 1) { - //call createCk method to get Ck + // call createCk method to get Ck val Ck: Array[Array[String]] = aprioriGen(L(k - 2), k) if (Ck != null) { - //call createLk method to get Lk + // call createLk method to get Lk val Lk: Array[(Array[String], Int)] = scanD( dataSet, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala index bd256bac10c0f..3e90f9f9dac2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala @@ -91,14 +91,14 @@ object AprioriByCartesian extends Logging with Serializable { minSupport: Double, sc: SparkContext): Array[(Set[String], Int)] = { - //dataSet length + // dataSet length val dataSetLen: Long = input.count() - //the count line for minSupport + // the count line for minSupport val minCount = minSupport * dataSetLen // This algorithm finds frequent item set, so convert each element of RDD to set val dataSet = input.map(_.toSet) - //definite L collection that using save all of frequent item set + // definite L collection that using save all of frequent item set val L = collection.mutable.ArrayBuffer[RDD[(Set[String], Int)]]() val L1: RDD[(Set[String], Int)] = aprioriStepOne(dataSet, minCount) @@ -121,7 +121,7 @@ object AprioriByCartesian extends Logging with Serializable { k = k + 1 L += Lk } - //return all result in L + // return all result in L val retArr = collection.mutable.ArrayBuffer[(Set[String], Int)]() for (l <- L) { retArr.appendAll(l.collect()) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala index fc2db8e32073e..025274009503f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala @@ -17,10 +17,12 @@ package org.apache.spark.mllib.fim -import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD import org.apache.spark.broadcast._ +import org.apache.spark.rdd.RDD +import org.apache.spark.{Logging, SparkContext} + +import scala.collection.mutable.{ArrayBuffer, Map} /** * calculate frequent item set using FPGrowth algorithm with dada set and minSupport @@ -47,18 +49,15 @@ class FPGrowth extends Logging with Serializable { val minCount = minSuport * count logDebug("minSuppot count:" + minSuport) - //one times scan data db to get L1 + // one times scan data db to get L1 val L1 = FPGStepOne(RDD, minCount) logDebug("L1 length:" + L1.length) logDebug("L1:" + L1) - //two times scan data db to get Ln + // two times scan data db to get Ln val Ln = FPGStepTwo(sc, RDD, minCount, L1) - //add L1 and Ln to get fim - val fim = L1 ++ Ln - - return fim - + // add L1 and Ln to get fim, and return it + L1 ++ Ln } /** @@ -90,10 +89,8 @@ class FPGrowth extends Logging with Serializable { RDD: RDD[Array[String]], minCount: Double, L1: Array[(String, Int)]): Array[(String, Int)] = { - //broadcast L1 + // broadcast L1 val bdL1 = sc.broadcast(L1) - //val bdL1List = bdL1.value - RDD.flatMap(line => L12LineMap(line, bdL1)) .groupByKey() .flatMap(line => FPTree(line, minCount)) @@ -117,16 +114,13 @@ class FPGrowth extends Logging with Serializable { // broadcast value val bdL1List = bdL1.value // the result variable - var lineArrayBuffer = collection.mutable.ArrayBuffer[(String, Int)]() + var lineArrayBuffer = ArrayBuffer[(String, Int)]() for (item <- line) { - val opt = bdL1List.find(_._1.equals(item)) - if (opt != None) { lineArrayBuffer ++= opt } - } // sort array @@ -135,15 +129,14 @@ class FPGrowth extends Logging with Serializable { .sortWith(_._2 > _._2) .toArray - - var arrArrayBuffer = collection.mutable.ArrayBuffer[(String, Array[String])]() + var arrArrayBuffer = ArrayBuffer[(String, Array[String])]() /** * give (a,4) (b 3),(c,3),after * b,((a,4) * c,((a,4) (b 3)) */ - var arrBuffer = collection.mutable.ArrayBuffer[String]() + var arrBuffer = ArrayBuffer[String]() for (item <- lineArray) { val arr = lineArray.take(lineArray.indexOf(item)) @@ -151,16 +144,13 @@ class FPGrowth extends Logging with Serializable { if (arr.length > 0) { for (tempArr <- arr) { - //remain key arrBuffer += tempArr._1 } arrArrayBuffer += ((item._1, arrBuffer.toArray)) } } - - return arrArrayBuffer.toArray - + arrArrayBuffer.toArray } /** @@ -175,8 +165,8 @@ class FPGrowth extends Logging with Serializable { // the set of construction CPFTree val value = line._2 - val _lineBuffer = collection.mutable.ArrayBuffer[(String, Int)]() - val map = scala.collection.mutable.Map[String, Int]() + val resultBuffer = ArrayBuffer[(String, Int)]() + val map = Map[String, Int]() // tree step var k = 1 // loop the data set while k>0 @@ -215,7 +205,7 @@ class FPGrowth extends Logging with Serializable { if (lineTemp.size != 0) { line = lineTemp.toArray.array - _lineBuffer ++= line + resultBuffer ++= line } } @@ -229,9 +219,7 @@ class FPGrowth extends Logging with Serializable { } } - - return _lineBuffer.toArray - + resultBuffer.toArray } } From 03df2b64663f6615d2c8f26329f10fb58313cd59 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 18 Jan 2015 22:07:02 -0800 Subject: [PATCH 05/10] refactory according to comments --- .../spark/mllib/fim/AprioriByBroadcast.scala | 242 ------------------ .../spark/mllib/fim/AprioriByCartesian.scala | 146 ----------- .../org/apache/spark/mllib/fim/FPGrowth.scala | 226 ---------------- .../apache/spark/mllib/fim/AprioriSuite.scala | 148 ----------- .../spark/mllib/fim/FPGrowthSuite.scala | 82 ------ 5 files changed, 844 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala deleted file mode 100644 index ca67c6069d0f3..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByBroadcast.scala +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.mllib.fim - -import org.apache.spark.SparkContext._ -import org.apache.spark.broadcast._ -import org.apache.spark.rdd.RDD -import org.apache.spark.{Logging, SparkContext} - -/** - * Calculate frequent item set using Apriori algorithm with minSupport. - * The apriori algorithm have two steps: - * step one is scan the data set to get L1 by minSuppprt - * step two is scan the data set multiple to get Lk - */ -object AprioriByBroadcast extends Logging with Serializable { - - /** - * Create C1 which contains all of single item in data Set. - * - * @param dataSet For mining frequent itemsets dataset - * @return single item in data Set - */ - def createC1(dataSet: RDD[Array[String]]): Array[Array[String]] = { - // get all distinct item in the RDD - val itemCollection = dataSet.flatMap(line => line).distinct().collect() - - // define new array which item is an array form - val itemArrCollection = collection.mutable.ArrayBuffer[Array[String]]() - - // change the itemsCollection into itemArrCollection - for (item <- itemCollection) { - itemArrCollection += Array[String](item) - } - - itemArrCollection.toArray - } - - - /** - * create Lk from Ck.Lk is generated by Ck when the frequent of Ck bigger than minCount - * @param dataSet For mining frequent itemsets dataset - * @param Ck Candidate set - * @param minCount The minimum degree of support - * @return Lk - */ - def scanD(dataSet: RDD[Array[String]], - Ck: Array[Array[String]], - minCount: Double, - sc: SparkContext): Array[(Array[String], Int)] = { - // broadcast Ck - val broadcastCk = sc.broadcast(Ck) - val Lk = dataSet.flatMap(line => containCk(line, broadcastCk)) - .filter(_.length > 0) - .map(v => (v, 1)) - .reduceByKey(_ + _) - .filter(_._2 >= (minCount)) - .map(v => (v._1.split(" "), v._2)) - .collect() - - Lk - } - - - /** - * containCk method. - * @param line dataset line - * @param broadcastCk L1 - * @return get Ck array - */ - def containCk(line: Array[String], - broadcastCk: Broadcast[Array[Array[String]]]): Array[String] = { - - // Ck broadcast value - val broadcastCkList: Array[Array[String]] = broadcastCk.value - // - var dbLineArrayBuffer = collection.mutable.ArrayBuffer[String]() - // the count number - var k: Int = 0 - - for (broadcastCk <- broadcastCkList) { - val bdArray: Array[String] = broadcastCk.sortWith((s, t) => s.compareTo(t) < 0).array - - if (bdArray.toSet subsetOf (line.toSet)) { - val bdString: String = bdArray.mkString(" ") - dbLineArrayBuffer ++= Array[(String)](bdString) - k = k + 1 - } - } - - if (k == 0) { - dbLineArrayBuffer ++= Array[(String)]("") - } - dbLineArrayBuffer.toArray.array - } - - /** - * create Ck by Lk - * @param Lk - * @param k - * @return Ck - */ - def aprioriGen(Lk: Array[(Array[String], Int)], - k: Int): Array[Array[String]] = { - - val LkLen = Lk.length - val CkBuffer = collection.mutable.ArrayBuffer[Array[String]]() - - // get Ck from Lk - for (i <- 0 to LkLen - 1) - for (j <- i + 1 to LkLen - 1) { - // get Lk:k-2 before k-2 item - val L1: Array[String] = - Lk(i)._1.take(k - 2).sortWith((s, t) => s.compareTo(t) < 0) - val L2: Array[String] = - Lk(j)._1.take(k - 2).sortWith((s, t) => s.compareTo(t) < 0) - - // merge set while the two set L1 and L2 equals - if (L1.mkString.equals(L2.mkString)) { - CkBuffer.append((Lk(i)._1.toSet ++ Lk(j)._1.toSet).toArray) - } - } - - if (CkBuffer.length > 0) { - CkBuffer.toArray.array - } else { - null - } - } - - /** - * create L1 - * @param dataSet For mining frequent item sets dataset - * @param minCount The minimum degree of support - * @return L1 - */ - def aprioriStepOne(dataSet: RDD[Array[String]], - minCount: Double): Array[(Array[String], Int)] = { - dataSet.flatMap(line => line) - .map(v => (v, 1)) - .reduceByKey(_ + _) - .filter(_._2 >= minCount) - .map(v => line2Array(v)) - .collect() - } - - /** - * change line type - * @param line line type (String,Int) - * @return line tpye (Array[String],Int) - */ - def line2Array(line: (String, Int)): (Array[String], Int) = { - (Array[String](line._1), line._2) - } - - /** - * apriori algorithm. - * Solving frequent item sets based on the data set and the minimum degree of support. - * The first phase, the scan time data sets, computing frequent item sets L1. - * The second stage, multiple scan data sets, computing frequent item sets Lk. - * @param dataSet For mining frequent item sets dataset - * @param minSupport The minimum degree of support - * @param sc - * @return frequent item sets - */ - def apriori(dataSet: RDD[Array[String]], - minSupport: Double, - sc: SparkContext): Array[(Set[String], Int)] = { - - // dataSet length - val dataSetLen: Long = dataSet.count() - // the count line for minSupport - val minCount = minSupport * dataSetLen - - // definite L collection that using save all of frequent item set - val L = collection.mutable.ArrayBuffer[Array[(Array[String], Int)]]() - val FIS = collection.mutable.ArrayBuffer[(Set[String], Int)]() - - // call aprioriStepOne method to get L1 - val L1: Array[(Array[String], Int)] = aprioriStepOne(dataSet, minCount) - logDebug("L1 length:" + L1.length) - logDebug("L1:" + L1) - - // L1 assignment to L - if (L1.length > 0) { - L += L1 - - for (arr <- L1) { - FIS.append((arr._1.toSet, arr._2)) - } - - // step counter - var k: Int = 2 - // do the loop while the k > 0 and L length > 1 - while ((k > 0) && L(k - 2).length > 1) { - - // call createCk method to get Ck - val Ck: Array[Array[String]] = aprioriGen(L(k - 2), k) - - if (Ck != null) { - // call createLk method to get Lk - val Lk: Array[(Array[String], Int)] = - scanD( - dataSet, - Ck, - minCount, - sc) - // Lk assignment to L - L += Lk - - for (arr <- Lk) { - FIS.append((arr._1.toSet, arr._2)) - } - - k = k + 1 - } - else { - k = -1 - } - } - FIS.toArray - } - else { - Array[(Set[String], Int)]() - } - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala deleted file mode 100644 index 3e90f9f9dac2a..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/AprioriByCartesian.scala +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.fim - -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD -import org.apache.spark.{Logging, SparkContext} - -/** - * Calculate frequent item set using Apriori algorithm with minSupport. - * The apriori algorithm have two steps: - * step one is scan the data set to get L1 by minSuppprt - * step two is scan the data set multiple to get Lk - */ -object AprioriByCartesian extends Logging with Serializable { - - /** - * create L1 - * @param dataSet For mining frequent item sets dataset - * @param minCount The minimum degree of support - * @return L1 - */ - def aprioriStepOne(dataSet: RDD[Set[String]], - minCount: Double): RDD[(Set[String], Int)] = { - dataSet.flatMap(line => line) - .map(v => (v, 1)) - .reduceByKey(_ + _) - .filter(_._2 >= minCount) - .map(x => (Set(x._1), x._2)) - } - - /** - * create Lk from Ck.Lk is generated by Ck when the frequent of Ck bigger than minSupport - * @param Ck Candidate set - * @param minCount The minimum degree of support - * @return Lk - */ - def scanD(dataSet: RDD[Set[String]], - Ck: RDD[Set[String]], - minCount: Double, - sc: SparkContext): RDD[(Set[String], Int)] = { - - dataSet.cartesian(Ck).map(x => - if (x._2.subsetOf(x._1)) { - (x._2, 1) - } else { - (x._2, 0) - }).reduceByKey(_ + _).filter(x => x._2 >= minCount) - } - - /** - * create Ck by Lk - * @param Lk - * @param k - * @return Ck - */ - def aprioriGen(Lk: RDD[Set[String]], - k: Int): RDD[Set[String]] = { - Lk.cartesian(Lk) - .map(x => x._1 ++ x._2) - .filter(x => x.size == k) - .distinct() - } - - /** - * apriori algorithm using cartesian of two RDD. - * Solving frequent item sets based on the data set and the minimum degree of support. - * The first phase, the scan time data sets, computing frequent item sets L1. - * The second stage, multiple scan data sets, computing frequent item sets Lk. - * @param input For mining frequent item sets dataset - * @param minSupport The minimum degree of support - * @param sc - * @return frequent item sets - */ - def apriori(input: RDD[Array[String]], - minSupport: Double, - sc: SparkContext): Array[(Set[String], Int)] = { - - // dataSet length - val dataSetLen: Long = input.count() - // the count line for minSupport - val minCount = minSupport * dataSetLen - // This algorithm finds frequent item set, so convert each element of RDD to set - val dataSet = input.map(_.toSet) - - // definite L collection that using save all of frequent item set - val L = collection.mutable.ArrayBuffer[RDD[(Set[String], Int)]]() - - val L1: RDD[(Set[String], Int)] = aprioriStepOne(dataSet, minCount) - if (L1.count() > 0) { - L += L1 - var Lk = L1 - - // step counter - var k = 2 - - while (L(k - 2).count() > 1) { - - // get candidate of frequent item set - val Ck: RDD[Set[String]] = aprioriGen(L(k - 2).map(x => x._1), k) - - // scan input data set to calculate degree of support for each candidate, - // and filter out the one not ineligible - Lk = scanD(dataSet, Ck, minCount, sc) - - k = k + 1 - L += Lk - } - // return all result in L - val retArr = collection.mutable.ArrayBuffer[(Set[String], Int)]() - for (l <- L) { - retArr.appendAll(l.collect()) - } - retArr.toArray - } else { - Array[(Set[String], Int)]() - } - } - - def printLk(Lk: RDD[(Set[String], Int)], k: Int) { - print("L" + (k - 2) + " size " + Lk.count() + " value: ") - Lk.collect().foreach(x => print("(" + x._1 + ", " + x._2 + ") ")) - println() - } - - def printCk(Ck: RDD[Set[String]], k: Int) { - print("C" + (k - 2) + " size " + Ck.count() + " value: ") - Ck.collect().foreach(print) - println() - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala deleted file mode 100644 index 025274009503f..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/fim/FPGrowth.scala +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.fim - -import org.apache.spark.SparkContext._ -import org.apache.spark.broadcast._ -import org.apache.spark.rdd.RDD -import org.apache.spark.{Logging, SparkContext} - -import scala.collection.mutable.{ArrayBuffer, Map} - -/** - * calculate frequent item set using FPGrowth algorithm with dada set and minSupport - * the FPGrowth algorithm have two step task - * step one is scaning data db to get L1 by minSuppprt - * step two is scan data db once to get Lk - */ -class FPGrowth extends Logging with Serializable { - - /** - * FPGrowth algorithm: - * step 1:calculate L1 by minSupport - * step 2: calculate Ln by FP-Tree - * @param sc sparkContext - * @param RDD For mining frequent item sets dataset - * @param minSuport The minimum degree of support - * @return frequent item sets - */ - def FPGrowth(RDD: RDD[Array[String]], - minSuport: Double, - sc: SparkContext): Array[(String, Int)] = { - val count = RDD.count() - logDebug("data set count:" + count) - val minCount = minSuport * count - logDebug("minSuppot count:" + minSuport) - - // one times scan data db to get L1 - val L1 = FPGStepOne(RDD, minCount) - logDebug("L1 length:" + L1.length) - logDebug("L1:" + L1) - - // two times scan data db to get Ln - val Ln = FPGStepTwo(sc, RDD, minCount, L1) - // add L1 and Ln to get fim, and return it - L1 ++ Ln - } - - /** - * Step 1: calculate L1 by min support - * @param RDD For mining frequent item sets dataset - * @param minCount The minimum degree of support - * @return L1 - */ - def FPGStepOne(RDD: RDD[Array[String]], - minCount: Double): Array[(String, Int)] = { - RDD.flatMap(v => v) - .map(v => (v, 1)) - .reduceByKey(_ + _) - .filter(_._2 >= minCount) - .collect() - .distinct - .sortWith(_._2 > _._2) - } - - /** - * step 2: using PFP-Tree to calculate the fim - * @param sc sparkContext - * @param RDD For mining frequent item sets dataset - * @param minCount The minimum degree of support - * @param L1 frenauent item set as length 1 - * @return Ln - */ - def FPGStepTwo(sc: SparkContext, - RDD: RDD[Array[String]], - minCount: Double, - L1: Array[(String, Int)]): Array[(String, Int)] = { - // broadcast L1 - val bdL1 = sc.broadcast(L1) - RDD.flatMap(line => L12LineMap(line, bdL1)) - .groupByKey() - .flatMap(line => FPTree(line, minCount)) - .collect() - - } - - /** - * create CFP-Tree - * give L1,example:a 2,b 4,c 3 and give line,example: a,b,c,d - * after calculate,the result is: - * a,(b,c) - * c,(b) - * note,the result have not b,() - * @param line dataset line - * @param bdL1 L1 - * @return CFP-Tree - */ - def L12LineMap(line: Array[String], - bdL1: Broadcast[Array[(String, Int)]]): Array[(String, Array[String])] = { - // broadcast value - val bdL1List = bdL1.value - // the result variable - var lineArrayBuffer = ArrayBuffer[(String, Int)]() - - for (item <- line) { - val opt = bdL1List.find(_._1.equals(item)) - if (opt != None) { - lineArrayBuffer ++= opt - } - } - - // sort array - val lineArray = lineArrayBuffer - .sortWith(_._1 > _._1) - .sortWith(_._2 > _._2) - .toArray - - var arrArrayBuffer = ArrayBuffer[(String, Array[String])]() - - /** - * give (a,4) (b 3),(c,3),after - * b,((a,4) - * c,((a,4) (b 3)) - */ - var arrBuffer = ArrayBuffer[String]() - for (item <- lineArray) { - val arr = lineArray.take(lineArray.indexOf(item)) - - arrBuffer.clear() - - if (arr.length > 0) { - for (tempArr <- arr) { - arrBuffer += tempArr._1 - } - arrArrayBuffer += ((item._1, arrBuffer.toArray)) - } - - } - arrArrayBuffer.toArray - } - - /** - * genarate fim set by FPTree,everyone node have a CPFTree that can combination frenquent item - * @param line dataset line - * @param minCount The minimum degree of support - * @return fim - */ - def FPTree(line: (String, Iterable[Array[String]]), minCount: Double): Array[(String, Int)] = { - // frequently item - val key = line._1 - // the set of construction CPFTree - val value = line._2 - - val resultBuffer = ArrayBuffer[(String, Int)]() - val map = Map[String, Int]() - // tree step - var k = 1 - // loop the data set while k>0 - while (k > 0) { - map.clear() - - // loop data set - for (it <- value) { - if (it.length >= k) { - // from m get n combinations,using scala method - val lineCom = it.toList.combinations(k).toList - - // add key to combination - for (item <- lineCom) { - // sort array - val list2key: List[String] = (item :+ key) - .sortWith(_ > _) - - val s = list2key.mkString(" ") - - if (map.get(s) == None) { - map(s) = 1 - } - else { - map(s) = map.apply(s) + 1 - } - } - } - } - - var line: Array[(String, Int)] = null - - if (map.size != 0) { - // get fim set - val lineTemp = map.filter(_._2 >= minCount) - - if (lineTemp.size != 0) { - line = lineTemp.toArray.array - resultBuffer ++= line - } - - } - - // reset k value - if ((line == null) || (line.length == 0)) { - k = 0 - } - else { - k = k + 1 - } - - } - resultBuffer.toArray - } - -} - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala deleted file mode 100644 index e08e6b6ffb41a..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/fim/AprioriSuite.scala +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.mllib.fim - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.rdd.RDD -import org.scalatest.FunSuite - -/** - * scala test unit - * using Practical Machine Learning Book data test the apriori algorithm result by minSupport from 0.9 to 0.1 - */ -class AprioriSuite extends FunSuite with LocalSparkContext { - - test("test FIM with AprioriByBroadcast dataset 1") - { - val arr = AprioriSuite.createFIMDataSet1() - val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByBroadcast.apriori - - val dataSet = sc.parallelize(arr) - val rdd = dataSet.map(line => line.split(" ")) - - for (i <- 1 to 9){ - println(s"frequent item set with support ${i/10d}") - target(rdd, i/10d, sc).foreach(x => print("(" + x._1 + "), ")) - println() - } - } - - test("test FIM with AprioriByBroadcast dataset 2") - { - val arr = AprioriSuite.createFIMDataSet2() - val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByBroadcast.apriori - - val dataSet = sc.parallelize(arr) - val rdd = dataSet.map(line => line.split(" ")) - - assert(target(rdd,0.9,sc).length == 0) - - assert(target(rdd,0.8,sc).length == 1) - - assert(target(rdd,0.7,sc).length == 1) - - assert(target(rdd,0.6,sc).length == 2) - - assert(target(rdd,0.5,sc).length == 18) - - assert(target(rdd,0.4,sc).length == 18) - - assert(target(rdd,0.3,sc).length == 54) - - assert(target(rdd,0.2,sc).length == 54) - - assert(target(rdd,0.1,sc).length == 625) - - } - - test("test FIM with AprioriByCartesian dataset 1") - { - val arr = AprioriSuite.createFIMDataSet1() - val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByCartesian.apriori - - val dataSet = sc.parallelize(arr) - val rdd = dataSet.map(line => line.split(" ")) - - for (i <- 1 to 9){ - println(s"frequent item set with support ${i/10d}") - target(rdd, i/10d, sc).foreach(x => print("(" + x._1 + "), ")) - println() - } - - - } - test("test FIM with AprioriByCartesian dataset 2") - { - val arr = AprioriSuite.createFIMDataSet2() - val target: (RDD[Array[String]], Double, SparkContext) => Array[(Set[String], Int)]= AprioriByCartesian.apriori - - val dataSet = sc.parallelize(arr) - val rdd = dataSet.map(line => line.split(" ")) - - assert(target(rdd,0.9,sc).length == 0) - - assert(target(rdd,0.8,sc).length == 1) - - assert(target(rdd,0.7,sc).length == 1) - - assert(target(rdd,0.6,sc).length == 2) - - assert(target(rdd,0.5,sc).length == 18) - - assert(target(rdd,0.4,sc).length == 18) - - assert(target(rdd,0.3,sc).length == 54) - - assert(target(rdd,0.2,sc).length == 54) - - assert(target(rdd,0.1,sc).length == 625) - - } - -} - -/** - * create dataset - */ -object AprioriSuite -{ - /** - * create dataset using Practical Machine Learning Book data - * @return dataset - */ - def createFIMDataSet1():Array[String] = { - val arr = Array[String]( - "1 3 4", - "2 3 5", - "1 2 3 5", - "2 5") - arr - } - - def createFIMDataSet2():Array[String] = { - val arr = Array[String]( - "r z h j p", - "z y x w v u t s", - "z", - "r x n o s", - "y r x z q t p", - "y z x e q s t m") - arr - } -} - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala deleted file mode 100644 index 5156babcd1975..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/fim/FPGrowthSuite.scala +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.mllib.fim - -import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext - -/** - * scala test unit - * using Practical Machine Learning Book data test the FPGrowth algorithm result by minSupport from 0.9 to 0.1 - */ -class FPGrowthSuite extends FunSuite with LocalSparkContext { - - test("test FIM with FPGrowth") - { - val arr = AprioriSuite.createFIMDataSet2() - - assert(arr.length === 6) - val dataSet = sc.parallelize(arr) - assert(dataSet.count() == 6) - val rdd = dataSet.map(line => line.split(" ")) - assert(rdd.count() == 6) - - //check frenquent item set length - val fimWithFPGrowth = new FPGrowth() - println(fimWithFPGrowth.FPGrowth(rdd,0.1,sc).length) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.9,sc).length == 0) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.8,sc).length == 1) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.7,sc).length == 1) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.6,sc).length == 2) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.5,sc).length == 18) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.4,sc).length == 18) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.3,sc).length == 54) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.2,sc).length == 54) - - assert(fimWithFPGrowth.FPGrowth(rdd,0.1,sc).length == 625) - - } -} - -/** - * create dataset - */ -object FPGrowthSuite -{ - /** - * create dataset using Practical Machine Learning Book data - * @return dataset - */ - def createFIMDataSet():Array[String] = - { - val arr = Array[String]("r z h j p", - "z y x w v u t s", - "z", - "r x n o s", - "y r x z q t p", - "y z x e q s t m") - return arr - } -} From eb3e4ca0709696b6b2b8afd1cfc56a5a9f87555d Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 18 Jan 2015 22:10:05 -0800 Subject: [PATCH 06/10] add FPGrowth --- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 208 ++++++++++++++++++ .../spark/mllib/fpm/FPGrowthModel.scala | 24 ++ .../spark/mllib/fpm/FPGrowthSuite.scala | 72 ++++++ 3 files changed, 304 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala new file mode 100644 index 0000000000000..b164e1843522a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fpm + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.broadcast._ +import org.apache.spark.rdd.RDD + +import scala.collection.mutable.{ArrayBuffer, Map} + +/** + * This class implements Parallel FPGrowth algorithm to do frequent pattern matching on input data. + * Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an + * independent group of mining tasks. More detail of this algorithm can be found at + * http://infolab.stanford.edu/~echang/recsys08-69.pdf + */ +class FPGrowth private(private var minSupport: Double) extends Logging with Serializable { + + /** + * Constructs a FPGrowth instance with default parameters: + * {minSupport: 0.5} + */ + def this() = this(0.5) + + /** + * set the minimal support level, default is 0.5 + * @param minSupport minimal support level + */ + def setMinSupport(minSupport: Double): this.type = { + this.minSupport = minSupport + this + } + + /** + * Compute a FPGrowth Model that contains frequent pattern result. + * @param data input data set + * @return FPGrowth Model + */ + def run(data: RDD[Array[String]]): FPGrowthModel = { + val model = runAlgorithm(data) + model + } + + /** + * Implementation of PFP. + */ + private def runAlgorithm(data: RDD[Array[String]]): FPGrowthModel = { + val count = data.count() + val minCount = minSupport * count + val single = generateSingleItem(data, minCount) + val combinations = generateCombinations(data, minCount, single) + new FPGrowthModel(single ++ combinations) + } + + /** + * Generate single item pattern by filtering the input data using minimal support level + */ + private def generateSingleItem( + data: RDD[Array[String]], + minCount: Double): Array[(String, Int)] = { + data.flatMap(v => v) + .map(v => (v, 1)) + .reduceByKey(_ + _) + .filter(_._2 >= minCount) + .collect() + .distinct + .sortWith(_._2 > _._2) + } + + /** + * Generate combination of items by computing on FPTree, + * the computation is done on each FPTree partitions. + */ + private def generateCombinations( + data: RDD[Array[String]], + minCount: Double, + singleItem: Array[(String, Int)]): Array[(String, Int)] = { + val single = data.context.broadcast(singleItem) + data.flatMap(basket => createFPTree(basket, single)) + .groupByKey() + .flatMap(partition => runFPTree(partition, minCount)) + .collect() + } + + /** + * Create FP-Tree partition for the giving basket + */ + private def createFPTree( + basket: Array[String], + singleItem: Broadcast[Array[(String, Int)]]): Array[(String, Array[String])] = { + var output = ArrayBuffer[(String, Array[String])]() + var combination = ArrayBuffer[String]() + val single = singleItem.value + var items = ArrayBuffer[(String, Int)]() + + // Filter the basket by single item pattern + val iterator = basket.iterator + while (iterator.hasNext){ + val item = iterator.next + val opt = single.find(_._1.equals(item)) + if (opt != None) { + items ++= opt + } + } + + // Sort it and create the item combinations + val sortedItems = items.sortWith(_._1 > _._1).sortWith(_._2 > _._2).toArray + val itemIterator = sortedItems.iterator + while (itemIterator.hasNext) { + combination.clear() + val item = itemIterator.next + val firstNItems = sortedItems.take(sortedItems.indexOf(item)) + if (firstNItems.length > 0) { + val iterator = firstNItems.iterator + while (iterator.hasNext) { + val elem = iterator.next + combination += elem._1 + } + output += ((item._1, combination.toArray)) + } + } + output.toArray + } + + /** + * Generate frequent pattern by walking through the FPTree + */ + private def runFPTree( + partition: (String, Iterable[Array[String]]), + minCount: Double): Array[(String, Int)] = { + val key = partition._1 + val value = partition._2 + val output = ArrayBuffer[(String, Int)]() + val map = Map[String, Int]() + + // Walk through the FPTree partition to generate all combinations that satisfy + // the minimal support level. + var k = 1 + while (k > 0) { + map.clear() + val iterator = value.iterator + while (iterator.hasNext) { + val pattern = iterator.next + if (pattern.length >= k) { + val combination = pattern.toList.combinations(k).toList + val itemIterator = combination.iterator + while (itemIterator.hasNext){ + val item = itemIterator.next + val list2key: List[String] = (item :+ key).sortWith(_ > _) + val newKey = list2key.mkString(" ") + if (map.get(newKey) == None) { + map(newKey) = 1 + } else { + map(newKey) = map.apply(newKey) + 1 + } + } + } + } + var eligible: Array[(String, Int)] = null + if (map.size != 0) { + val candidate = map.filter(_._2 >= minCount) + if (candidate.size != 0) { + eligible = candidate.toArray + output ++= eligible + } + } + if ((eligible == null) || (eligible.length == 0)) { + k = 0 + } else { + k = k + 1 + } + } + output.toArray + } +} + +/** + * Top-level methods for calling FPGrowth. + */ +object FPGrowth{ + + /** + * Generate a FPGrowth Model using the given minimal support level. + * + * @param data input baskets stored as `RDD[Array[String]]` + * @param minSupport minimal support level, for example 0.5 + */ + def train(data: RDD[Array[String]], minSupport: Double): FPGrowthModel = { + new FPGrowth().setMinSupport(minSupport).run(data) + } +} + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala new file mode 100644 index 0000000000000..cb3348d654733 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.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.spark.mllib.fpm + +/** + * A FPGrowth Model for FPGrowth, each element is a frequent pattern with count. + */ +class FPGrowthModel (val frequentPattern: Array[(String, Int)]) extends Serializable { +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala new file mode 100644 index 0000000000000..aad5b4a3f2afb --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.mllib.fpm + +import org.scalatest.FunSuite +import org.apache.spark.mllib.util.LocalSparkContext + +class FPGrowthSuite extends FunSuite with LocalSparkContext { + + test("test FPGrowth algorithm") + { + val arr = FPGrowthSuite.createTestData() + + assert(arr.length === 6) + val dataSet = sc.parallelize(arr) + assert(dataSet.count() == 6) + val rdd = dataSet.map(line => line.split(" ")) + assert(rdd.count() == 6) + + val algorithm = new FPGrowth() + algorithm.setMinSupport(0.9) + assert(algorithm.run(rdd).frequentPattern.length == 0) + algorithm.setMinSupport(0.8) + assert(algorithm.run(rdd).frequentPattern.length == 1) + algorithm.setMinSupport(0.7) + assert(algorithm.run(rdd).frequentPattern.length == 1) + algorithm.setMinSupport(0.6) + assert(algorithm.run(rdd).frequentPattern.length == 2) + algorithm.setMinSupport(0.5) + assert(algorithm.run(rdd).frequentPattern.length == 18) + algorithm.setMinSupport(0.4) + assert(algorithm.run(rdd).frequentPattern.length == 18) + algorithm.setMinSupport(0.3) + assert(algorithm.run(rdd).frequentPattern.length == 54) + algorithm.setMinSupport(0.2) + assert(algorithm.run(rdd).frequentPattern.length == 54) + algorithm.setMinSupport(0.1) + assert(algorithm.run(rdd).frequentPattern.length == 625) + } +} + +object FPGrowthSuite +{ + /** + * Create test data set + */ + def createTestData():Array[String] = + { + val arr = Array[String]( + "r z h k p", + "z y x w v u t s", + "s x o n r", + "x z y m t s q e", + "z", + "x z y r q t p") + arr + } +} From d110ab2883b0aa85cf0c94f6ca2f12cc0ea73a37 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 19 Jan 2015 00:08:01 -0800 Subject: [PATCH 07/10] change test case to use MLlibTestSparkContext --- .../test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index aad5b4a3f2afb..e29399ffe71fe 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.mllib.fpm import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.MLlibTestSparkContext -class FPGrowthSuite extends FunSuite with LocalSparkContext { +class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { test("test FPGrowth algorithm") { From 93f3280fb1b9897f40b695683824aef619a5b8c2 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 31 Jan 2015 00:50:36 +0800 Subject: [PATCH 08/10] create FPTree class --- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 156 ++++-------- .../spark/mllib/fpm/FPGrowthModel.scala | 2 +- .../org/apache/spark/mllib/fpm/FPTree.scala | 240 ++++++++++++++++++ .../spark/mllib/fpm/FPGrowthSuite.scala | 54 +++- .../apache/spark/mllib/fpm/FPTreeSuite.scala | 145 +++++++++++ 5 files changed, 480 insertions(+), 117 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index b164e1843522a..71e66392d64ba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -17,29 +17,34 @@ package org.apache.spark.mllib.fpm +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.broadcast.Broadcast import org.apache.spark.Logging -import org.apache.spark.SparkContext._ -import org.apache.spark.broadcast._ import org.apache.spark.rdd.RDD -import scala.collection.mutable.{ArrayBuffer, Map} + /** - * This class implements Parallel FPGrowth algorithm to do frequent pattern matching on input data. + * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data. * Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an * independent group of mining tasks. More detail of this algorithm can be found at - * http://infolab.stanford.edu/~echang/recsys08-69.pdf + * [[http://dx.doi.org/10.1145/1454008.1454027, PFP]], and the original FP-growth paper can be found at + * [[http://dx.doi.org/10.1145/335191.335372, FP-growth]] + * + * @param minSupport the minimal support level of the frequent pattern, any pattern appears more than + * (minSupport * size-of-the-dataset) times will be output */ class FPGrowth private(private var minSupport: Double) extends Logging with Serializable { /** * Constructs a FPGrowth instance with default parameters: - * {minSupport: 0.5} + * {minSupport: 0.3} */ - def this() = this(0.5) + def this() = this(0.3) /** - * set the minimal support level, default is 0.5 + * set the minimal support level, default is 0.3 * @param minSupport minimal support level */ def setMinSupport(minSupport: Double): this.type = { @@ -49,87 +54,82 @@ class FPGrowth private(private var minSupport: Double) extends Logging with Seri /** * Compute a FPGrowth Model that contains frequent pattern result. - * @param data input data set + * @param data input data set, each element contains a transaction * @return FPGrowth Model */ def run(data: RDD[Array[String]]): FPGrowthModel = { - val model = runAlgorithm(data) - model - } - - /** - * Implementation of PFP. - */ - private def runAlgorithm(data: RDD[Array[String]]): FPGrowthModel = { val count = data.count() val minCount = minSupport * count val single = generateSingleItem(data, minCount) val combinations = generateCombinations(data, minCount, single) - new FPGrowthModel(single ++ combinations) + val all = single.map(v => (Array[String](v._1), v._2)).union(combinations) + new FPGrowthModel(all.collect()) } /** * Generate single item pattern by filtering the input data using minimal support level + * @return array of frequent pattern with its count */ private def generateSingleItem( data: RDD[Array[String]], - minCount: Double): Array[(String, Int)] = { - data.flatMap(v => v) - .map(v => (v, 1)) + minCount: Double): RDD[(String, Long)] = { + val single = data.flatMap(v => v.toSet) + .map(v => (v, 1L)) .reduceByKey(_ + _) .filter(_._2 >= minCount) - .collect() - .distinct - .sortWith(_._2 > _._2) + .sortBy(_._2) + single } /** - * Generate combination of items by computing on FPTree, + * Generate combination of frequent pattern by computing on FPTree, * the computation is done on each FPTree partitions. + * @return array of frequent pattern with its count */ private def generateCombinations( data: RDD[Array[String]], minCount: Double, - singleItem: Array[(String, Int)]): Array[(String, Int)] = { - val single = data.context.broadcast(singleItem) - data.flatMap(basket => createFPTree(basket, single)) - .groupByKey() - .flatMap(partition => runFPTree(partition, minCount)) - .collect() + singleItem: RDD[(String, Long)]): RDD[(Array[String], Long)] = { + val single = data.context.broadcast(singleItem.collect()) + data.flatMap(transaction => createConditionPatternBase(transaction, single)) + .aggregateByKey(new FPTree)( + (aggregator, condPattBase) => aggregator.add(condPattBase), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) + .flatMap(partition => partition._2.mine(minCount, partition._1)) } /** * Create FP-Tree partition for the giving basket + * @return an array contains a tuple, whose first element is the single + * item (hash key) and second element is its condition pattern base */ - private def createFPTree( - basket: Array[String], - singleItem: Broadcast[Array[(String, Int)]]): Array[(String, Array[String])] = { + private def createConditionPatternBase( + transaction: Array[String], + singleBC: Broadcast[Array[(String, Long)]]): Array[(String, Array[String])] = { var output = ArrayBuffer[(String, Array[String])]() var combination = ArrayBuffer[String]() - val single = singleItem.value - var items = ArrayBuffer[(String, Int)]() - - // Filter the basket by single item pattern - val iterator = basket.iterator - while (iterator.hasNext){ - val item = iterator.next - val opt = single.find(_._1.equals(item)) - if (opt != None) { - items ++= opt - } - } - - // Sort it and create the item combinations - val sortedItems = items.sortWith(_._1 > _._1).sortWith(_._2 > _._2).toArray - val itemIterator = sortedItems.iterator + var items = ArrayBuffer[(String, Long)]() + val single = singleBC.value + val singleMap = single.toMap + + // Filter the basket by single item pattern and sort + // by single item and its count + val candidates = transaction + .filter(singleMap.contains) + .map(item => (item, singleMap(item))) + .sortBy(_._1) + .sortBy(_._2) + .toArray + + val itemIterator = candidates.iterator while (itemIterator.hasNext) { combination.clear() - val item = itemIterator.next - val firstNItems = sortedItems.take(sortedItems.indexOf(item)) + val item = itemIterator.next() + val firstNItems = candidates.take(candidates.indexOf(item)) if (firstNItems.length > 0) { val iterator = firstNItems.iterator while (iterator.hasNext) { - val elem = iterator.next + val elem = iterator.next() combination += elem._1 } output += ((item._1, combination.toArray)) @@ -138,56 +138,6 @@ class FPGrowth private(private var minSupport: Double) extends Logging with Seri output.toArray } - /** - * Generate frequent pattern by walking through the FPTree - */ - private def runFPTree( - partition: (String, Iterable[Array[String]]), - minCount: Double): Array[(String, Int)] = { - val key = partition._1 - val value = partition._2 - val output = ArrayBuffer[(String, Int)]() - val map = Map[String, Int]() - - // Walk through the FPTree partition to generate all combinations that satisfy - // the minimal support level. - var k = 1 - while (k > 0) { - map.clear() - val iterator = value.iterator - while (iterator.hasNext) { - val pattern = iterator.next - if (pattern.length >= k) { - val combination = pattern.toList.combinations(k).toList - val itemIterator = combination.iterator - while (itemIterator.hasNext){ - val item = itemIterator.next - val list2key: List[String] = (item :+ key).sortWith(_ > _) - val newKey = list2key.mkString(" ") - if (map.get(newKey) == None) { - map(newKey) = 1 - } else { - map(newKey) = map.apply(newKey) + 1 - } - } - } - } - var eligible: Array[(String, Int)] = null - if (map.size != 0) { - val candidate = map.filter(_._2 >= minCount) - if (candidate.size != 0) { - eligible = candidate.toArray - output ++= eligible - } - } - if ((eligible == null) || (eligible.length == 0)) { - k = 0 - } else { - k = k + 1 - } - } - output.toArray - } } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala index cb3348d654733..1f490d6ccdd59 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala @@ -20,5 +20,5 @@ package org.apache.spark.mllib.fpm /** * A FPGrowth Model for FPGrowth, each element is a frequent pattern with count. */ -class FPGrowthModel (val frequentPattern: Array[(String, Int)]) extends Serializable { +class FPGrowthModel (val frequentPattern: Array[(Array[String], Long)]) extends Serializable { } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala new file mode 100644 index 0000000000000..2dc2631d55232 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fpm + +import scala.collection.mutable.{ListBuffer, ArrayBuffer, Map} + +class FPTree extends Serializable { + + val root: FPTreeNode = new FPTreeNode(null, 0) + + def add(transaction: Array[String]): this.type = { + var index = 0 + val size = transaction.size + var curr = root + while (index < size) { + if (curr.children.contains(transaction(index))) { + val node = curr.children(transaction(index)) + node.count = node.count + 1 + curr = node + } else { + val newNode = new FPTreeNode(transaction(index), 1) + newNode.parent = curr + curr.children(transaction(index)) = newNode + curr = newNode + } + index = index + 1 + } + + // TODO: in oder to further reduce the amount of data for shuffle, + // remove the same pattern which has the same hash number + this + } + + /** + * merge with the input tree + * @param tree the tree to merge + * @return tree after merge + */ + def merge(tree: FPTree): this.type = { + // merge two trees recursively to remove all duplicated nodes + mergeTree(this.root, tree.root) + this + } + + /** + * merge two trees from their root node + * @param tree1 root node of the tree one + * @param tree2 root node of the tree two + * @return root node after merge + */ + private def mergeTree(tree1: FPTreeNode, tree2: FPTreeNode): FPTreeNode = { + // firstly merge two roots, then iterate on the second tree, merge all children of it to the first tree + require(tree1 != null) + require(tree2 != null) + if (!tree2.isRoot) { + require(tree1.item.equals(tree2.item)) + tree1.count = tree1.count + tree2.count + } + if (!tree2.isLeaf) { + val it = tree2.children.iterator + while (it.hasNext) { + val node = mergeSubTree(tree1, it.next()._2) + tree1.children(node.item) = node + node.parent = tree1 + } + } + tree1 + } + + /** + * merge the second tree into the children of the first tree, if there is a match + * @param tree1Root root node of the tree one + * @param subTree2 the child of the tree two + * @return root node after merge + */ + private def mergeSubTree(tree1Root: FPTreeNode, subTree2: FPTreeNode): FPTreeNode = { + if (tree1Root.children.contains(subTree2.item)) { + mergeTree(tree1Root.children(subTree2.item), subTree2) + } else { + subTree2 + } + } + + /** + * Generate all frequent patterns by mining the FPTree recursively + * @param minCount minimal count + * @param suffix key of this tree + * @return + */ + def mine(minCount: Double, suffix: String): Array[(Array[String], Long)] = { + val condPattBase = expandFPTree(this) + mineFPTree(condPattBase, minCount, suffix) + } + + /** + * This function will walk through the tree and build all conditional pattern base out of it + * @param tree the tree to expand + * @return conditional pattern base + */ + private def expandFPTree(tree: FPTree): ArrayBuffer[ArrayBuffer[String]] = { + var output: ArrayBuffer[ArrayBuffer[String]] = null + if (!tree.root.isLeaf) { + val it = tree.root.children.iterator + while (it.hasNext) { + val childOuput = expandFPTreeNode(it.next()._2) + if (output == null) output = childOuput else output ++= childOuput + } + } + output + } + + /** + * Expand from the input node + * @param node tree node + * @return conditional pattern base + */ + private def expandFPTreeNode(node: FPTreeNode): ArrayBuffer[ArrayBuffer[String]] = { + // Iterate on all children and build the output recursively + val output = new ArrayBuffer[ArrayBuffer[String]]() + for (i <- 0 to node.count - 1) { + output.append(ArrayBuffer[String](node.item)) + } + val it = node.children.iterator + var i = 0 + while (it.hasNext) { + val child = it.next() + val childOutput = expandFPTreeNode(child._2) + require(childOutput.size <= output.size) + for (buffer <- childOutput) { + output(i) ++= buffer + i = i + 1 + } + } + output + } + + /** + * Generate all frequent patterns by combinations of condition pattern base. + * This implementation is different from classical fp-growth algorithm which generate + * FPTree recursively. + * + * @param condPattBase condition pattern base + * @param minCount the minimum count + * @param suffix key of the condition pattern base + * @return frequent item set + */ + private def mineFPTree( + condPattBase: ArrayBuffer[ArrayBuffer[String]], + minCount: Double, + suffix: String): Array[(Array[String], Long)] = { + // frequently item + val key = suffix + // the set of construction CPFTree + val value = condPattBase + + // tree step.start 2th + var k = 1 + // save all frequently item set + val fimSetBuffer = ArrayBuffer[(String, Long)]() + // save step k's lineComList temp value to next step k+1 compute combinations + var lineComListTempBuffer = ArrayBuffer[String]() + // loop the data set from 1 to k while k>0 + while (k > 0) { + // save step k's lineComList temp value + var lineComListBuffer = ListBuffer[List[String]]() + // loop every value to combinations while each value length >= k + for (v <- value) { + val vLen = v.length + if (vLen >= k) { + // calculate each value combinations while each value k == 2 + if (k == 1) { + val lineCom = v.toList.combinations(k) + lineComListBuffer ++= lineCom.toList + } else { + /* if each value length > k,it need calculate the intersect of each value & before combinations */ + val union_lineComListTemp2v = v intersect lineComListTempBuffer.toArray.array + // calculate each value combinations after intersect + if (union_lineComListTemp2v.length >= k) { + val lineCom = union_lineComListTemp2v.toList.combinations(k) + lineComListBuffer ++= lineCom.toList + } + } + } + } + + var lineComList: Array[(String, Long)] = null + // reset + lineComListTempBuffer = ArrayBuffer[String]() + // calculate frequent item set + if (lineComListBuffer != null || lineComListBuffer.size != 0) { + val lineComListTemp = lineComListBuffer + .map( v => ( (v :+ key).sortWith(_ > _),1) ) + .groupBy(_._1) + .map(v => (v._1,v._2.length)) + .filter(_._2 >= minCount) + if ( lineComListTemp != null || lineComListTemp.size != 0) { + lineComList = lineComListTemp + .map(v => (v._1.mkString(" "), v._2.toLong)) + .toArray + fimSetBuffer ++= lineComList + for (lcl <- lineComList) { + lineComListTempBuffer ++= lcl._1.split(" ") + } + } + } + // reset k value + if (lineComList == null || lineComList.length == 0) { + k = 0 + } else { + k = k + 1 + } + } + val fimSetArray = fimSetBuffer + .map(v => (v._1.split(" "), v._2)) + .toArray + fimSetArray + } +} + +class FPTreeNode(val item: String, var count: Int) extends Serializable { + var parent: FPTreeNode = null + val children: Map[String, FPTreeNode] = Map[String, FPTreeNode]() + def isLeaf: Boolean = children.size == 0 + def isRoot: Boolean = parent == null +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index e29399ffe71fe..02181bf2b83e8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -17,15 +17,16 @@ package org.apache.spark.mllib.fpm import org.scalatest.FunSuite + import org.apache.spark.mllib.util.MLlibTestSparkContext -class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { +class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { test("test FPGrowth algorithm") { - val arr = FPGrowthSuite.createTestData() + val arr = FPGrowthSuite.createFIMDataSet() - assert(arr.length === 6) + assert(arr.length == 6) val dataSet = sc.parallelize(arr) assert(dataSet.count() == 6) val rdd = dataSet.map(line => line.split(" ")) @@ -58,15 +59,42 @@ object FPGrowthSuite /** * Create test data set */ - def createTestData():Array[String] = - { - val arr = Array[String]( - "r z h k p", - "z y x w v u t s", - "s x o n r", - "x z y m t s q e", - "z", - "x z y r q t p") - arr + def createFIMDataSet():Array[String] = + { + val arr = Array[String]( + "r z h k p", + "z y x w v u t s", + "s x o n r", + "x z y m t s q e", + "z", + "x z y r q t p") + arr + } + + def printTree(tree: FPTree) = printTreeRoot(tree.root, 0) + + private def printTreeRoot(tree: FPTreeNode, level: Int): Unit = { + printNode(tree, level) + if (tree.isLeaf) return + val it = tree.children.iterator + while (it.hasNext) { + val child = it.next() + printTreeRoot(child._2, level + 1) + } + } + + private def printNode(node: FPTreeNode, level: Int) = { + for (i <- 0 to level) { + print("\t") + } + println(node.item + " " + node.count) + } + + def printFrequentPattern(pattern: Array[(Array[String], Long)]) = { + for (a <- pattern) { + a._1.foreach(x => print(x + " ")) + print(a._2) + println } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala new file mode 100644 index 0000000000000..92d415da1ddb7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.fpm + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.scalatest.FunSuite + +class FPTreeSuite extends FunSuite with MLlibTestSparkContext { + + test("add transaction to tree") { + val tree = new FPTree + tree.add(Array[String]("a", "b", "c")) + tree.add(Array[String]("a", "b", "y")) + tree.add(Array[String]("b")) + FPGrowthSuite.printTree(tree) + + assert(tree.root.children.size == 2) + assert(tree.root.children.contains("a")) + assert(tree.root.children("a").item.equals("a")) + assert(tree.root.children("a").count == 2) + assert(tree.root.children.contains("b")) + assert(tree.root.children("b").item.equals("b")) + assert(tree.root.children("b").count == 1) + var child = tree.root.children("a") + assert(child.children.size == 1) + assert(child.children.contains("b")) + assert(child.children("b").item.equals("b")) + assert(child.children("b").count == 2) + child = child.children("b") + assert(child.children.size == 2) + assert(child.children.contains("c")) + assert(child.children.contains("y")) + assert(child.children("c").item.equals("c")) + assert(child.children("y").item.equals("y")) + assert(child.children("c").count == 1) + assert(child.children("y").count == 1) + } + + test("merge tree") { + val tree1 = new FPTree + tree1.add(Array[String]("a", "b", "c")) + tree1.add(Array[String]("a", "b", "y")) + tree1.add(Array[String]("b")) + FPGrowthSuite.printTree(tree1) + + val tree2 = new FPTree + tree2.add(Array[String]("a", "b")) + tree2.add(Array[String]("a", "b", "c")) + tree2.add(Array[String]("a", "b", "c", "d")) + tree2.add(Array[String]("a", "x")) + tree2.add(Array[String]("a", "x", "y")) + tree2.add(Array[String]("c", "n")) + tree2.add(Array[String]("c", "m")) + FPGrowthSuite.printTree(tree2) + + val tree3 = tree1.merge(tree2) + FPGrowthSuite.printTree(tree3) + + assert(tree3.root.children.size == 3) + assert(tree3.root.children("a").count == 7) + assert(tree3.root.children("b").count == 1) + assert(tree3.root.children("c").count == 2) + val child1 = tree3.root.children("a") + assert(child1.children.size == 2) + assert(child1.children("b").count == 5) + assert(child1.children("x").count == 2) + val child2 = child1.children("b") + assert(child2.children.size == 2) + assert(child2.children("y").count == 1) + assert(child2.children("c").count == 3) + val child3 = child2.children("c") + assert(child3.children.size == 1) + assert(child3.children("d").count == 1) + val child4 = child1.children("x") + assert(child4.children.size == 1) + assert(child4.children("y").count == 1) + val child5 = tree3.root.children("c") + assert(child5.children.size == 2) + assert(child5.children("n").count == 1) + assert(child5.children("m").count == 1) + } + + /* + test("expand tree") { + val tree = new FPTree + tree.add(Array[String]("a", "b", "c")) + tree.add(Array[String]("a", "b", "y")) + tree.add(Array[String]("a", "b")) + tree.add(Array[String]("a")) + tree.add(Array[String]("b")) + tree.add(Array[String]("b", "n")) + + FPGrowthSuite.printTree(tree) + val buffer = tree.expandFPTree(tree) + for (a <- buffer) { + a.foreach(x => print(x + " ")) + println + } + } + */ + + test("mine tree") { + val tree = new FPTree + tree.add(Array[String]("a", "b", "c")) + tree.add(Array[String]("a", "b", "y")) + tree.add(Array[String]("a", "b")) + tree.add(Array[String]("a")) + tree.add(Array[String]("b")) + tree.add(Array[String]("b", "n")) + + FPGrowthSuite.printTree(tree) + val buffer = tree.mine(3.0, "t") + + for (a <- buffer) { + a._1.foreach(x => print(x + " ")) + print(a._2) + println + } + val s1 = buffer(0)._1 + val s2 = buffer(1)._1 + val s3 = buffer(2)._1 + assert(s1(1).equals("a")) + assert(s2(1).equals("b")) + assert(s3(1).equals("b")) + assert(s3(2).equals("a")) + assert(buffer(0)._2 == 4) + assert(buffer(1)._2 == 5) + assert(buffer(2)._2 == 3) + } +} From ec21f7dfcad6191e0c2d6d7fd93ac77012098e6c Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 31 Jan 2015 00:56:39 +0800 Subject: [PATCH 09/10] fix scalastyle --- .../main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala | 8 ++++---- .../main/scala/org/apache/spark/mllib/fpm/FPTree.scala | 9 ++++++--- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 71e66392d64ba..8f8a24f2462d1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -29,11 +29,11 @@ import org.apache.spark.rdd.RDD * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data. * Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an * independent group of mining tasks. More detail of this algorithm can be found at - * [[http://dx.doi.org/10.1145/1454008.1454027, PFP]], and the original FP-growth paper can be found at - * [[http://dx.doi.org/10.1145/335191.335372, FP-growth]] + * [[http://dx.doi.org/10.1145/1454008.1454027, PFP]], and the original FP-growth paper can be + * found at [[http://dx.doi.org/10.1145/335191.335372, FP-growth]] * - * @param minSupport the minimal support level of the frequent pattern, any pattern appears more than - * (minSupport * size-of-the-dataset) times will be output + * @param minSupport the minimal support level of the frequent pattern, any pattern appears + * more than (minSupport * size-of-the-dataset) times will be output */ class FPGrowth private(private var minSupport: Double) extends Logging with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala index 2dc2631d55232..ef37bf17cb351 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -64,7 +64,8 @@ class FPTree extends Serializable { * @return root node after merge */ private def mergeTree(tree1: FPTreeNode, tree2: FPTreeNode): FPTreeNode = { - // firstly merge two roots, then iterate on the second tree, merge all children of it to the first tree + // firstly merge two roots, then iterate on the second tree, merge all + // children of it to the first tree require(tree1 != null) require(tree2 != null) if (!tree2.isRoot) { @@ -108,7 +109,8 @@ class FPTree extends Serializable { } /** - * This function will walk through the tree and build all conditional pattern base out of it + * This function will walk through the tree and build all conditional pattern + * base out of it * @param tree the tree to expand * @return conditional pattern base */ @@ -187,7 +189,8 @@ class FPTree extends Serializable { val lineCom = v.toList.combinations(k) lineComListBuffer ++= lineCom.toList } else { - /* if each value length > k,it need calculate the intersect of each value & before combinations */ + // if each value length > k,it need calculate the intersect of each + // value & before combinations val union_lineComListTemp2v = v intersect lineComListTempBuffer.toArray.array // calculate each value combinations after intersect if (union_lineComListTemp2v.length >= k) { From 7e697256f4f1aef909618f259f147c4b8682d7f7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 1 Feb 2015 01:36:07 -0800 Subject: [PATCH 10/10] simplify FPTree and update FPGrowth --- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 180 +++++------ .../spark/mllib/fpm/FPGrowthModel.scala | 24 -- .../org/apache/spark/mllib/fpm/FPTree.scala | 281 ++++++------------ .../spark/mllib/fpm/FPGrowthSuite.scala | 99 +++--- .../apache/spark/mllib/fpm/FPTreeSuite.scala | 104 +++---- 5 files changed, 251 insertions(+), 437 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 8f8a24f2462d1..9591c7966e06a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -17,13 +17,15 @@ package org.apache.spark.mllib.fpm -import scala.collection.mutable.ArrayBuffer +import java.{util => ju} -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.Logging -import org.apache.spark.rdd.RDD +import scala.collection.mutable +import org.apache.spark.{SparkException, HashPartitioner, Logging, Partitioner} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +class FPGrowthModel(val freqItemsets: RDD[(Array[String], Long)]) extends Serializable /** * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data. @@ -34,18 +36,20 @@ import org.apache.spark.rdd.RDD * * @param minSupport the minimal support level of the frequent pattern, any pattern appears * more than (minSupport * size-of-the-dataset) times will be output + * @param numPartitions number of partitions used by parallel FP-growth */ -class FPGrowth private(private var minSupport: Double) extends Logging with Serializable { +class FPGrowth private ( + private var minSupport: Double, + private var numPartitions: Int) extends Logging with Serializable { /** * Constructs a FPGrowth instance with default parameters: - * {minSupport: 0.3} + * {minSupport: 0.3, numPartitions: auto} */ - def this() = this(0.3) + def this() = this(0.3, -1) /** - * set the minimal support level, default is 0.3 - * @param minSupport minimal support level + * Sets the minimal support level (default: 0.3). */ def setMinSupport(minSupport: Double): this.type = { this.minSupport = minSupport @@ -53,106 +57,106 @@ class FPGrowth private(private var minSupport: Double) extends Logging with Seri } /** - * Compute a FPGrowth Model that contains frequent pattern result. + * Sets the number of partitions used by parallel FP-growth (default: same as input data). + */ + def setNumPartitions(numPartitions: Int): this.type = { + this.numPartitions = numPartitions + this + } + + /** + * Computes an FP-Growth model that contains frequent itemsets. * @param data input data set, each element contains a transaction - * @return FPGrowth Model + * @return an [[FPGrowthModel]] */ def run(data: RDD[Array[String]]): FPGrowthModel = { + if (data.getStorageLevel == StorageLevel.NONE) { + logWarning("Input data is not cached.") + } val count = data.count() - val minCount = minSupport * count - val single = generateSingleItem(data, minCount) - val combinations = generateCombinations(data, minCount, single) - val all = single.map(v => (Array[String](v._1), v._2)).union(combinations) - new FPGrowthModel(all.collect()) + val minCount = math.ceil(minSupport * count).toLong + val numParts = if (numPartitions > 0) numPartitions else data.partitions.length + val partitioner = new HashPartitioner(numParts) + val freqItems = genFreqItems(data, minCount, partitioner) + val freqItemsets = genFreqItemsets(data, minCount, freqItems, partitioner) + new FPGrowthModel(freqItemsets) } /** - * Generate single item pattern by filtering the input data using minimal support level - * @return array of frequent pattern with its count + * Generates frequent items by filtering the input data using minimal support level. + * @param minCount minimum count for frequent itemsets + * @param partitioner partitioner used to distribute items + * @return array of frequent pattern ordered by their frequencies */ - private def generateSingleItem( + private def genFreqItems( data: RDD[Array[String]], - minCount: Double): RDD[(String, Long)] = { - val single = data.flatMap(v => v.toSet) - .map(v => (v, 1L)) - .reduceByKey(_ + _) + minCount: Long, + partitioner: Partitioner): Array[String] = { + data.flatMap { t => + val uniq = t.toSet + if (t.length != uniq.size) { + throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.") + } + t + }.map(v => (v, 1L)) + .reduceByKey(partitioner, _ + _) .filter(_._2 >= minCount) - .sortBy(_._2) - single + .collect() + .sortBy(-_._2) + .map(_._1) } /** - * Generate combination of frequent pattern by computing on FPTree, - * the computation is done on each FPTree partitions. - * @return array of frequent pattern with its count + * Generate frequent itemsets by building FP-Trees, the extraction is done on each partition. + * @param data transactions + * @param minCount minimum count for frequent itemsets + * @param freqItems frequent items + * @param partitioner partitioner used to distribute transactions + * @return an RDD of (frequent itemset, count) */ - private def generateCombinations( + private def genFreqItemsets( data: RDD[Array[String]], - minCount: Double, - singleItem: RDD[(String, Long)]): RDD[(Array[String], Long)] = { - val single = data.context.broadcast(singleItem.collect()) - data.flatMap(transaction => createConditionPatternBase(transaction, single)) - .aggregateByKey(new FPTree)( - (aggregator, condPattBase) => aggregator.add(condPattBase), - (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) - .flatMap(partition => partition._2.mine(minCount, partition._1)) + minCount: Long, + freqItems: Array[String], + partitioner: Partitioner): RDD[(Array[String], Long)] = { + val itemToRank = freqItems.zipWithIndex.toMap + data.flatMap { transaction => + genCondTransactions(transaction, itemToRank, partitioner) + }.aggregateByKey(new FPTree[Int], partitioner.numPartitions)( + (tree, transaction) => tree.add(transaction, 1L), + (tree1, tree2) => tree1.merge(tree2)) + .flatMap { case (part, tree) => + tree.extract(minCount, x => partitioner.getPartition(x) == part) + }.map { case (ranks, count) => + (ranks.map(i => freqItems(i)).toArray, count) + } } /** - * Create FP-Tree partition for the giving basket - * @return an array contains a tuple, whose first element is the single - * item (hash key) and second element is its condition pattern base + * Generates conditional transactions. + * @param transaction a transaction + * @param itemToRank map from item to their rank + * @param partitioner partitioner used to distribute transactions + * @return a map of (target partition, conditional transaction) */ - private def createConditionPatternBase( + private def genCondTransactions( transaction: Array[String], - singleBC: Broadcast[Array[(String, Long)]]): Array[(String, Array[String])] = { - var output = ArrayBuffer[(String, Array[String])]() - var combination = ArrayBuffer[String]() - var items = ArrayBuffer[(String, Long)]() - val single = singleBC.value - val singleMap = single.toMap - - // Filter the basket by single item pattern and sort - // by single item and its count - val candidates = transaction - .filter(singleMap.contains) - .map(item => (item, singleMap(item))) - .sortBy(_._1) - .sortBy(_._2) - .toArray - - val itemIterator = candidates.iterator - while (itemIterator.hasNext) { - combination.clear() - val item = itemIterator.next() - val firstNItems = candidates.take(candidates.indexOf(item)) - if (firstNItems.length > 0) { - val iterator = firstNItems.iterator - while (iterator.hasNext) { - val elem = iterator.next() - combination += elem._1 - } - output += ((item._1, combination.toArray)) + itemToRank: Map[String, Int], + partitioner: Partitioner): mutable.Map[Int, Array[Int]] = { + val output = mutable.Map.empty[Int, Array[Int]] + // Filter the basket by frequent items pattern and sort their ranks. + val filtered = transaction.flatMap(itemToRank.get) + ju.Arrays.sort(filtered) + val n = filtered.length + var i = n - 1 + while (i >= 0) { + val item = filtered(i) + val part = partitioner.getPartition(item) + if (!output.contains(part)) { + output(part) = filtered.slice(0, i + 1) } + i -= 1 } - output.toArray + output } - } - -/** - * Top-level methods for calling FPGrowth. - */ -object FPGrowth{ - - /** - * Generate a FPGrowth Model using the given minimal support level. - * - * @param data input baskets stored as `RDD[Array[String]]` - * @param minSupport minimal support level, for example 0.5 - */ - def train(data: RDD[Array[String]], minSupport: Double): FPGrowthModel = { - new FPGrowth().setMinSupport(minSupport).run(data) - } -} - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala deleted file mode 100644 index 1f490d6ccdd59..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowthModel.scala +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.fpm - -/** - * A FPGrowth Model for FPGrowth, each element is a frequent pattern with count. - */ -class FPGrowthModel (val frequentPattern: Array[(Array[String], Long)]) extends Serializable { -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala index ef37bf17cb351..1d2d777c00793 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -17,227 +17,118 @@ package org.apache.spark.mllib.fpm -import scala.collection.mutable.{ListBuffer, ArrayBuffer, Map} +import scala.collection.mutable +import scala.collection.mutable.ListBuffer -class FPTree extends Serializable { +/** + * FP-Tree data structure used in FP-Growth. + * @tparam T item type + */ +private[fpm] class FPTree[T] extends Serializable { + + import FPTree._ - val root: FPTreeNode = new FPTreeNode(null, 0) + val root: Node[T] = new Node(null) - def add(transaction: Array[String]): this.type = { - var index = 0 - val size = transaction.size + private val summaries: mutable.Map[T, Summary[T]] = mutable.Map.empty + + /** Adds a transaction with count. */ + def add(t: Iterable[T], count: Long = 1L): this.type = { + require(count > 0) var curr = root - while (index < size) { - if (curr.children.contains(transaction(index))) { - val node = curr.children(transaction(index)) - node.count = node.count + 1 - curr = node - } else { - val newNode = new FPTreeNode(transaction(index), 1) - newNode.parent = curr - curr.children(transaction(index)) = newNode - curr = newNode - } - index = index + 1 + curr.count += count + t.foreach { item => + val summary = summaries.getOrElseUpdate(item, new Summary) + summary.count += count + val child = curr.children.getOrElseUpdate(item, { + val newNode = new Node(curr) + newNode.item = item + summary.nodes += newNode + newNode + }) + child.count += count + curr = child } - - // TODO: in oder to further reduce the amount of data for shuffle, - // remove the same pattern which has the same hash number this } - /** - * merge with the input tree - * @param tree the tree to merge - * @return tree after merge - */ - def merge(tree: FPTree): this.type = { - // merge two trees recursively to remove all duplicated nodes - mergeTree(this.root, tree.root) + /** Merges another FP-Tree. */ + def merge(other: FPTree[T]): this.type = { + other.transactions.foreach { case (t, c) => + add(t, c) + } this } - /** - * merge two trees from their root node - * @param tree1 root node of the tree one - * @param tree2 root node of the tree two - * @return root node after merge - */ - private def mergeTree(tree1: FPTreeNode, tree2: FPTreeNode): FPTreeNode = { - // firstly merge two roots, then iterate on the second tree, merge all - // children of it to the first tree - require(tree1 != null) - require(tree2 != null) - if (!tree2.isRoot) { - require(tree1.item.equals(tree2.item)) - tree1.count = tree1.count + tree2.count - } - if (!tree2.isLeaf) { - val it = tree2.children.iterator - while (it.hasNext) { - val node = mergeSubTree(tree1, it.next()._2) - tree1.children(node.item) = node - node.parent = tree1 + /** Gets a subtree with the suffix. */ + private def project(suffix: T): FPTree[T] = { + val tree = new FPTree[T] + if (summaries.contains(suffix)) { + val summary = summaries(suffix) + summary.nodes.foreach { node => + var t = List.empty[T] + var curr = node.parent + while (!curr.isRoot) { + t = curr.item :: t + curr = curr.parent + } + tree.add(t, node.count) } } - tree1 + tree } - /** - * merge the second tree into the children of the first tree, if there is a match - * @param tree1Root root node of the tree one - * @param subTree2 the child of the tree two - * @return root node after merge - */ - private def mergeSubTree(tree1Root: FPTreeNode, subTree2: FPTreeNode): FPTreeNode = { - if (tree1Root.children.contains(subTree2.item)) { - mergeTree(tree1Root.children(subTree2.item), subTree2) - } else { - subTree2 - } - } + /** Returns all transactions in an iterator. */ + def transactions: Iterator[(List[T], Long)] = getTransactions(root) - /** - * Generate all frequent patterns by mining the FPTree recursively - * @param minCount minimal count - * @param suffix key of this tree - * @return - */ - def mine(minCount: Double, suffix: String): Array[(Array[String], Long)] = { - val condPattBase = expandFPTree(this) - mineFPTree(condPattBase, minCount, suffix) - } - - /** - * This function will walk through the tree and build all conditional pattern - * base out of it - * @param tree the tree to expand - * @return conditional pattern base - */ - private def expandFPTree(tree: FPTree): ArrayBuffer[ArrayBuffer[String]] = { - var output: ArrayBuffer[ArrayBuffer[String]] = null - if (!tree.root.isLeaf) { - val it = tree.root.children.iterator - while (it.hasNext) { - val childOuput = expandFPTreeNode(it.next()._2) - if (output == null) output = childOuput else output ++= childOuput + /** Returns all transactions under this node. */ + private def getTransactions(node: Node[T]): Iterator[(List[T], Long)] = { + var count = node.count + node.children.iterator.flatMap { case (item, child) => + getTransactions(child).map { case (t, c) => + count -= c + (item :: t, c) } - } - output - } - - /** - * Expand from the input node - * @param node tree node - * @return conditional pattern base - */ - private def expandFPTreeNode(node: FPTreeNode): ArrayBuffer[ArrayBuffer[String]] = { - // Iterate on all children and build the output recursively - val output = new ArrayBuffer[ArrayBuffer[String]]() - for (i <- 0 to node.count - 1) { - output.append(ArrayBuffer[String](node.item)) - } - val it = node.children.iterator - var i = 0 - while (it.hasNext) { - val child = it.next() - val childOutput = expandFPTreeNode(child._2) - require(childOutput.size <= output.size) - for (buffer <- childOutput) { - output(i) ++= buffer - i = i + 1 + } ++ { + if (count > 0) { + Iterator.single((Nil, count)) + } else { + Iterator.empty } } - output } - - /** - * Generate all frequent patterns by combinations of condition pattern base. - * This implementation is different from classical fp-growth algorithm which generate - * FPTree recursively. - * - * @param condPattBase condition pattern base - * @param minCount the minimum count - * @param suffix key of the condition pattern base - * @return frequent item set - */ - private def mineFPTree( - condPattBase: ArrayBuffer[ArrayBuffer[String]], - minCount: Double, - suffix: String): Array[(Array[String], Long)] = { - // frequently item - val key = suffix - // the set of construction CPFTree - val value = condPattBase - - // tree step.start 2th - var k = 1 - // save all frequently item set - val fimSetBuffer = ArrayBuffer[(String, Long)]() - // save step k's lineComList temp value to next step k+1 compute combinations - var lineComListTempBuffer = ArrayBuffer[String]() - // loop the data set from 1 to k while k>0 - while (k > 0) { - // save step k's lineComList temp value - var lineComListBuffer = ListBuffer[List[String]]() - // loop every value to combinations while each value length >= k - for (v <- value) { - val vLen = v.length - if (vLen >= k) { - // calculate each value combinations while each value k == 2 - if (k == 1) { - val lineCom = v.toList.combinations(k) - lineComListBuffer ++= lineCom.toList - } else { - // if each value length > k,it need calculate the intersect of each - // value & before combinations - val union_lineComListTemp2v = v intersect lineComListTempBuffer.toArray.array - // calculate each value combinations after intersect - if (union_lineComListTemp2v.length >= k) { - val lineCom = union_lineComListTemp2v.toList.combinations(k) - lineComListBuffer ++= lineCom.toList - } - } - } - } - var lineComList: Array[(String, Long)] = null - // reset - lineComListTempBuffer = ArrayBuffer[String]() - // calculate frequent item set - if (lineComListBuffer != null || lineComListBuffer.size != 0) { - val lineComListTemp = lineComListBuffer - .map( v => ( (v :+ key).sortWith(_ > _),1) ) - .groupBy(_._1) - .map(v => (v._1,v._2.length)) - .filter(_._2 >= minCount) - if ( lineComListTemp != null || lineComListTemp.size != 0) { - lineComList = lineComListTemp - .map(v => (v._1.mkString(" "), v._2.toLong)) - .toArray - fimSetBuffer ++= lineComList - for (lcl <- lineComList) { - lineComListTempBuffer ++= lcl._1.split(" ") + /** Extracts all patterns with valid suffix and minimum count. */ + def extract( + minCount: Long, + validateSuffix: T => Boolean = _ => true): Iterator[(List[T], Long)] = { + summaries.iterator.flatMap { case (item, summary) => + if (validateSuffix(item) && summary.count >= minCount) { + Iterator.single((item :: Nil, summary.count)) ++ + project(item).extract(minCount).map { case (t, c) => + (item :: t, c) } - } - } - // reset k value - if (lineComList == null || lineComList.length == 0) { - k = 0 } else { - k = k + 1 + Iterator.empty } } - val fimSetArray = fimSetBuffer - .map(v => (v._1.split(" "), v._2)) - .toArray - fimSetArray } } -class FPTreeNode(val item: String, var count: Int) extends Serializable { - var parent: FPTreeNode = null - val children: Map[String, FPTreeNode] = Map[String, FPTreeNode]() - def isLeaf: Boolean = children.size == 0 - def isRoot: Boolean = parent == null +private[fpm] object FPTree { + + /** Representing a node in an FP-Tree. */ + class Node[T](val parent: Node[T]) extends Serializable { + var item: T = _ + var count: Long = 0L + val children: mutable.Map[T, Node[T]] = mutable.Map.empty + + def isRoot: Boolean = parent == null + } + + /** Summary of a item in an FP-Tree. */ + private class Summary[T] extends Serializable { + var count: Long = 0L + val nodes: ListBuffer[Node[T]] = ListBuffer.empty + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index 02181bf2b83e8..71ef60da6dd32 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -22,79 +22,52 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { - test("test FPGrowth algorithm") - { - val arr = FPGrowthSuite.createFIMDataSet() - - assert(arr.length == 6) - val dataSet = sc.parallelize(arr) - assert(dataSet.count() == 6) - val rdd = dataSet.map(line => line.split(" ")) - assert(rdd.count() == 6) - - val algorithm = new FPGrowth() - algorithm.setMinSupport(0.9) - assert(algorithm.run(rdd).frequentPattern.length == 0) - algorithm.setMinSupport(0.8) - assert(algorithm.run(rdd).frequentPattern.length == 1) - algorithm.setMinSupport(0.7) - assert(algorithm.run(rdd).frequentPattern.length == 1) - algorithm.setMinSupport(0.6) - assert(algorithm.run(rdd).frequentPattern.length == 2) - algorithm.setMinSupport(0.5) - assert(algorithm.run(rdd).frequentPattern.length == 18) - algorithm.setMinSupport(0.4) - assert(algorithm.run(rdd).frequentPattern.length == 18) - algorithm.setMinSupport(0.3) - assert(algorithm.run(rdd).frequentPattern.length == 54) - algorithm.setMinSupport(0.2) - assert(algorithm.run(rdd).frequentPattern.length == 54) - algorithm.setMinSupport(0.1) - assert(algorithm.run(rdd).frequentPattern.length == 625) - } -} - -object FPGrowthSuite -{ - /** - * Create test data set - */ - def createFIMDataSet():Array[String] = - { - val arr = Array[String]( + test("FP-Growth") { + val transactions = Seq( "r z h k p", "z y x w v u t s", "s x o n r", "x z y m t s q e", "z", "x z y r q t p") - arr - } + .map(_.split(" ")) + val rdd = sc.parallelize(transactions, 2).cache() - def printTree(tree: FPTree) = printTreeRoot(tree.root, 0) + val fpg = new FPGrowth() - private def printTreeRoot(tree: FPTreeNode, level: Int): Unit = { - printNode(tree, level) - if (tree.isLeaf) return - val it = tree.children.iterator - while (it.hasNext) { - val child = it.next() - printTreeRoot(child._2, level + 1) - } - } + val model6 = fpg + .setMinSupport(0.9) + .setNumPartitions(1) + .run(rdd) + assert(model6.freqItemsets.count() === 0) - private def printNode(node: FPTreeNode, level: Int) = { - for (i <- 0 to level) { - print("\t") + val model3 = fpg + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd) + val freqItemsets3 = model3.freqItemsets.collect().map { case (items, count) => + (items.toSet, count) } - println(node.item + " " + node.count) - } + val expected = Set( + (Set("s"), 3L), (Set("z"), 5L), (Set("x"), 4L), (Set("t"), 3L), (Set("y"), 3L), + (Set("r"), 3L), + (Set("x", "z"), 3L), (Set("t", "y"), 3L), (Set("t", "x"), 3L), (Set("s", "x"), 3L), + (Set("y", "x"), 3L), (Set("y", "z"), 3L), (Set("t", "z"), 3L), + (Set("y", "x", "z"), 3L), (Set("t", "x", "z"), 3L), (Set("t", "y", "z"), 3L), + (Set("t", "y", "x"), 3L), + (Set("t", "y", "x", "z"), 3L)) + assert(freqItemsets3.toSet === expected) - def printFrequentPattern(pattern: Array[(Array[String], Long)]) = { - for (a <- pattern) { - a._1.foreach(x => print(x + " ")) - print(a._2) - println - } + val model2 = fpg + .setMinSupport(0.3) + .setNumPartitions(4) + .run(rdd) + assert(model2.freqItemsets.count() === 54) + + val model1 = fpg + .setMinSupport(0.1) + .setNumPartitions(8) + .run(rdd) + assert(model1.freqItemsets.count() === 625) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala index 92d415da1ddb7..04017f67c311d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.mllib.fpm -import org.apache.spark.mllib.util.MLlibTestSparkContext +import scala.language.existentials + import org.scalatest.FunSuite +import org.apache.spark.mllib.util.MLlibTestSparkContext + class FPTreeSuite extends FunSuite with MLlibTestSparkContext { - test("add transaction to tree") { - val tree = new FPTree - tree.add(Array[String]("a", "b", "c")) - tree.add(Array[String]("a", "b", "y")) - tree.add(Array[String]("b")) - FPGrowthSuite.printTree(tree) + test("add transaction") { + val tree = new FPTree[String] + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "y")) + .add(Seq("b")) assert(tree.root.children.size == 2) assert(tree.root.children.contains("a")) @@ -52,24 +54,21 @@ class FPTreeSuite extends FunSuite with MLlibTestSparkContext { } test("merge tree") { - val tree1 = new FPTree - tree1.add(Array[String]("a", "b", "c")) - tree1.add(Array[String]("a", "b", "y")) - tree1.add(Array[String]("b")) - FPGrowthSuite.printTree(tree1) + val tree1 = new FPTree[String] + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "y")) + .add(Seq("b")) - val tree2 = new FPTree - tree2.add(Array[String]("a", "b")) - tree2.add(Array[String]("a", "b", "c")) - tree2.add(Array[String]("a", "b", "c", "d")) - tree2.add(Array[String]("a", "x")) - tree2.add(Array[String]("a", "x", "y")) - tree2.add(Array[String]("c", "n")) - tree2.add(Array[String]("c", "m")) - FPGrowthSuite.printTree(tree2) + val tree2 = new FPTree[String] + .add(Seq("a", "b")) + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "c", "d")) + .add(Seq("a", "x")) + .add(Seq("a", "x", "y")) + .add(Seq("c", "n")) + .add(Seq("c", "m")) val tree3 = tree1.merge(tree2) - FPGrowthSuite.printTree(tree3) assert(tree3.root.children.size == 3) assert(tree3.root.children("a").count == 7) @@ -95,51 +94,22 @@ class FPTreeSuite extends FunSuite with MLlibTestSparkContext { assert(child5.children("m").count == 1) } - /* - test("expand tree") { - val tree = new FPTree - tree.add(Array[String]("a", "b", "c")) - tree.add(Array[String]("a", "b", "y")) - tree.add(Array[String]("a", "b")) - tree.add(Array[String]("a")) - tree.add(Array[String]("b")) - tree.add(Array[String]("b", "n")) - - FPGrowthSuite.printTree(tree) - val buffer = tree.expandFPTree(tree) - for (a <- buffer) { - a.foreach(x => print(x + " ")) - println - } - } - */ - - test("mine tree") { - val tree = new FPTree - tree.add(Array[String]("a", "b", "c")) - tree.add(Array[String]("a", "b", "y")) - tree.add(Array[String]("a", "b")) - tree.add(Array[String]("a")) - tree.add(Array[String]("b")) - tree.add(Array[String]("b", "n")) - - FPGrowthSuite.printTree(tree) - val buffer = tree.mine(3.0, "t") + test("extract freq itemsets") { + val tree = new FPTree[String] + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "y")) + .add(Seq("a", "b")) + .add(Seq("a")) + .add(Seq("b")) + .add(Seq("b", "n")) - for (a <- buffer) { - a._1.foreach(x => print(x + " ")) - print(a._2) - println - } - val s1 = buffer(0)._1 - val s2 = buffer(1)._1 - val s3 = buffer(2)._1 - assert(s1(1).equals("a")) - assert(s2(1).equals("b")) - assert(s3(1).equals("b")) - assert(s3(2).equals("a")) - assert(buffer(0)._2 == 4) - assert(buffer(1)._2 == 5) - assert(buffer(2)._2 == 3) + val freqItemsets = tree.extract(3L).map { case (items, count) => + (items.toSet, count) + }.toSet + val expected = Set( + (Set("a"), 4L), + (Set("b"), 5L), + (Set("a", "b"), 3L)) + assert(freqItemsets === expected) } }