You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by ro...@apache.org on 2009/10/19 00:19:57 UTC

svn commit: r826539 [2/2] - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/common/ core/src/main/java/org/apache/mahout/common/cache/ core/src/main/java/org/apache/mahout/fpm/ core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ core/src/...

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPGrowth.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPGrowth.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPGrowth.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPGrowth.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,613 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth.fpgrowth;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.Map.Entry;
+import org.apache.commons.lang.mutable.MutableLong;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.mahout.common.Pair;
+import org.apache.mahout.fpm.pfpgrowth.convertors.TopKPatternsOutputConvertor;
+import org.apache.mahout.fpm.pfpgrowth.convertors.TransactionIterator;
+import org.apache.mahout.fpm.pfpgrowth.convertors.string.TopKStringPatterns;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * PFPGrowth Class has both vanilla FPGrowth and Top K FPGrowth
+ * 
+ * @param <AttributePrimitive>
+ */
+public class FPGrowth<AttributePrimitive extends Comparable<? super AttributePrimitive>> {
+
+  private static final Logger log = LoggerFactory.getLogger(FPGrowth.class);
+
+  /**
+   * Create FPTree with node counts incremented by addCount variable given the
+   * root node and the List of Attributes in transaction sorted by support
+   * 
+   * @param TransactionTree object to which the transaction has to be added to
+   * @param myList List of transactions sorted by support
+   * @param addCount amount by which the Node count has to be incremented
+   * @param minSupport the MutableLong value which contains the current value(dynamic) of support
+   * @param attributeFrequency the list of attributes and their frequency
+   * @return the number of new nodes added
+   */
+  final private int treeAddCount(FPTree tree, int[] myList, int addCount,
+      MutableLong minSupport, long[] attributeFrequency) {
+    int temp = FPTree.ROOTNODEID;
+    int ret = 0;
+    boolean addCountMode = true;
+    int child = -1;
+    for (int attribute : myList) {
+      if (attributeFrequency[attribute] < minSupport.intValue())
+        return ret;
+      if (addCountMode) {
+        child = tree.childWithAttribute(temp, attribute);
+        if (child == -1) {
+          addCountMode = false;
+        } else {
+          tree.addCount(child, addCount);
+          temp = child;
+        }
+      }
+      if (!addCountMode) {
+        child = tree.createNode(temp, attribute, addCount);
+        temp = child;
+        ret++;
+      }
+    }
+
+    return ret;
+
+  }
+
+  private FrequentPatternMaxHeap generateSinglePathPatterns(FPTree tree, int K,
+      MutableLong minSupportMutable) {
+    FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(K);
+
+    int tempNode = FPTree.ROOTNODEID;
+    Pattern frequentItem = new Pattern();
+    while (true) {
+      if (tree.childCount(tempNode) == 0) {
+        break;
+      } else {
+        if (tree.childCount(tempNode) > 1) {
+          log.info("This should not happen {} {}", tree.childCount(tempNode),
+              tempNode);
+        }
+        tempNode = tree.childAtIndex(tempNode, 0);
+        if (tree.count(tempNode) < minSupportMutable.intValue()) {
+          continue;
+        }
+        frequentItem.add(tree.attribute(tempNode), tree.count(tempNode));
+      }
+    }
+    if (frequentItem.length > 0) {
+      frequentPatterns.insert(frequentItem);
+    }
+
+    return frequentPatterns;
+  }
+
+  private void traverseAndBuildConditionalFPTreeData(int firstConditionalNode,
+      MutableLong minSupportMutable, FPTree conditionalTree, FPTree tree) {
+
+    // Build Subtable
+    int conditionalNode = firstConditionalNode;
+
+    while (conditionalNode != -1) {
+      long nextNodeCount = tree.count(conditionalNode);
+      int pathNode = tree.parent(conditionalNode);
+      int prevConditional = -1;
+
+      while (0 != pathNode) { // dummy root node
+        int attribute = tree.attribute(pathNode);
+        if (tree.getHeaderSupportCount(attribute) < minSupportMutable
+            .intValue()) {
+          pathNode = tree.parent(pathNode);
+          continue;
+        }
+        // update and increment the headerTable Counts
+        conditionalTree.addHeaderCount(attribute, nextNodeCount);
+
+        int conditional = tree.conditional(pathNode);
+        // if its a new conditional tree node
+
+        if (conditional == 0) {
+          tree.setConditional(pathNode, conditionalTree.createConditionalNode(
+              attribute, 0));
+          conditional = tree.conditional(pathNode);
+          conditionalTree.addHeaderNext(attribute, conditional);
+        } else {
+          conditionalTree.setSinglePath(false);
+        }
+
+        if (prevConditional != -1) { // if there is a child element
+          conditionalTree.setParent(prevConditional, conditional);
+        }
+
+        conditionalTree.addCount(conditional, nextNodeCount);
+        prevConditional = conditional;
+
+        pathNode = tree.parent(pathNode);
+
+      }
+      if (prevConditional != -1) {
+        conditionalTree.setParent(prevConditional, FPTree.ROOTNODEID);
+        if (conditionalTree.childCount(FPTree.ROOTNODEID) > 1
+            && conditionalTree.singlePath()) {
+          conditionalTree.setSinglePath(false);
+
+        }
+      }
+      conditionalNode = tree.next(conditionalNode);
+    }
+    tree.clearConditional();
+    conditionalTree.reorderHeaderTable();
+
+  }
+
+  final private FrequentPatternMaxHeap mergeHeap(
+      FrequentPatternMaxHeap frequentPatterns,
+      FrequentPatternMaxHeap returnedPatterns, int attribute, long count,
+      boolean addAttribute, boolean subPatternCheck) {
+
+    TreeSet<Pattern> myHeap = returnedPatterns.getHeap();
+    Iterator<Pattern> it = myHeap.iterator();
+
+    while (it.hasNext()) {
+      Pattern pattern = it.next();
+      long support = Math.min(count, pattern.getSupport());
+      if (frequentPatterns.addable(support)) {
+        pattern.add(attribute, count);
+        frequentPatterns.insert(pattern, subPatternCheck);
+      }
+    }
+    if (frequentPatterns.addable(count) && addAttribute) {
+      Pattern p = new Pattern();
+      p.add(attribute, count);
+      frequentPatterns.insert(p, subPatternCheck);
+    }
+    return frequentPatterns;
+  }
+
+  /**
+   * Top K FpGrowth Algorithm
+   * 
+   * @param TransactionTree to be mined
+   * @param minSupportMutable minimum support of the pattern to keep
+   * @param K Number of top frequent patterns to keep
+   * @param requiredFeatures Set of integer id's of features to mine
+   * @param outputCollector the Collector class which converts the given
+   *        frequent pattern in integer to AttributePrimitive
+   * @return Top K Frequent Patterns for each feature and their support
+   * @throws IOException
+   */
+  final private Map<Integer, FrequentPatternMaxHeap> fpGrowth(FPTree tree,
+      MutableLong minSupportMutable, int K, Set<Integer> requiredFeatures,
+      TopKPatternsOutputConvertor<AttributePrimitive> outputCollector)
+      throws IOException {
+
+    int minSupportValue = minSupportMutable.intValue();
+
+    Map<Integer, FrequentPatternMaxHeap> Patterns = new HashMap<Integer, FrequentPatternMaxHeap>();
+    FPTreeDepthCache treeCache = new FPTreeDepthCache();
+    for (int i = tree.getHeaderTableCount() - 1; i >= 0; i--) {
+      int attribute = tree.getAttributeAtIndex(i);
+      if (requiredFeatures.contains(attribute) == false)
+        continue;
+      log.info("Mining FTree Tree for all patterns with {}", attribute);
+      MutableLong minSupport = new MutableLong(minSupportValue);
+      FrequentPatternMaxHeap frequentPatterns = growth(tree, minSupport, K,
+          treeCache, 0, attribute);
+      Patterns.put(attribute, frequentPatterns);
+      outputCollector.collect(attribute, frequentPatterns);
+
+      minSupportValue = Math.max(minSupportValue, minSupport.intValue() / 2);
+      log.info("Found {} Patterns with Least Support {}", Patterns
+          .get(attribute).count, Patterns.get(attribute).least.support);
+    }
+    log.info("Tree Cache: First Level: Cache hits={} Cache Misses={}",
+        treeCache.getHits(), treeCache.getMisses());
+    return Patterns;
+  }
+
+  final private FrequentPatternMaxHeap growth(FPTree tree,
+      MutableLong minSupportMutable, int K, FPTreeDepthCache treeCache,
+      int level, int currentAttribute) {
+    FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(K);
+    FrequentPatternMaxHeap returnedPatterns = null;
+
+    int i = Arrays.binarySearch(tree.getHeaderTableAttributes(),
+        currentAttribute);
+    if (i < 0)
+      return frequentPatterns;
+
+    for (int j = tree.getHeaderTableCount(); i < j; i++) {
+      int attribute = tree.getAttributeAtIndex(i);
+      long count = tree.getHeaderSupportCount(attribute);
+      if (count < minSupportMutable.intValue())
+        continue;
+
+      FPTree conditionalTree = treeCache.getFirstLevelTree(attribute);
+      if (conditionalTree.isEmpty()) {
+        traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+            minSupportMutable, conditionalTree, tree);
+      }
+
+      if (attribute == currentAttribute) {
+
+        returnedPatterns = growthTopDown(conditionalTree, minSupportMutable, K,
+            treeCache, level + 1, true, currentAttribute);
+        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+            attribute, count, true, true);
+      } else {
+        returnedPatterns = growthTopDown(conditionalTree, minSupportMutable, K,
+            treeCache, level + 1, false, currentAttribute);
+
+        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+            attribute, count, false, true);
+      }
+      if (frequentPatterns.isFull()) {
+        if (minSupportMutable.intValue() < frequentPatterns.leastSupport()) {
+          minSupportMutable.setValue(frequentPatterns.leastSupport());
+        }
+      }
+
+    }
+
+    return frequentPatterns;
+  }
+
+  final private FrequentPatternMaxHeap growthTopDown(FPTree tree,
+      MutableLong minSupportMutable, int K, FPTreeDepthCache treeCache,
+      int level, boolean conditionalOfCurrentAttribute, int currentAttribute) {
+    FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(K);
+    FrequentPatternMaxHeap returnedPatterns = null;
+    if (conditionalOfCurrentAttribute == false) {
+      int index = Arrays.binarySearch(tree.getHeaderTableAttributes(),
+          currentAttribute);
+      if (index < 0)
+        return frequentPatterns;
+      else {
+        int attribute = tree.getAttributeAtIndex(index);
+        long count = tree.getHeaderSupportCount(attribute);
+        if (count < minSupportMutable.intValue())
+          return frequentPatterns;
+      }
+    }
+    if (tree.singlePath()) {
+      return generateSinglePathPatterns(tree, K, minSupportMutable);
+    }
+    for (int i = 0; i < tree.getHeaderTableCount(); i++) {
+      int attribute = tree.getAttributeAtIndex(i);
+      long count = tree.getHeaderSupportCount(attribute);
+      if (count < minSupportMutable.longValue())
+        continue;
+
+      FPTree conditionalTree = treeCache.getTree(level);
+
+      if (conditionalOfCurrentAttribute == true) {
+        traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+            minSupportMutable, conditionalTree, tree);
+        returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+            K, treeCache, level + 1, true, currentAttribute);
+        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+            attribute, count, true, true);
+
+      } else {
+        if (attribute == currentAttribute) {
+          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+              minSupportMutable, conditionalTree, tree);
+          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+              K, treeCache, level + 1, true, currentAttribute);
+          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+              attribute, count, true, false);
+
+        } else if (attribute > currentAttribute) {
+          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+              minSupportMutable, conditionalTree, tree);
+          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+              K, treeCache, level + 1, false, currentAttribute);
+          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+              attribute, count, false, true);
+
+        }
+      }
+      if (frequentPatterns.isFull()) {
+        if (minSupportMutable.intValue() < frequentPatterns.leastSupport()) {
+          minSupportMutable.setValue(frequentPatterns.leastSupport());
+        }
+      }
+    }
+
+    return frequentPatterns;
+  }
+
+  final private FrequentPatternMaxHeap growthBottomUp(FPTree tree,
+      MutableLong minSupportMutable, int K, FPTreeDepthCache treeCache,
+      int level, boolean conditionalOfCurrentAttribute, int currentAttribute) {
+    FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(K);
+    FrequentPatternMaxHeap returnedPatterns = null;
+    if (conditionalOfCurrentAttribute == false) {
+      int index = Arrays.binarySearch(tree.getHeaderTableAttributes(),
+          currentAttribute);
+      if (index < 0)
+        return frequentPatterns;
+      else {
+        int attribute = tree.getAttributeAtIndex(index);
+        long count = tree.getHeaderSupportCount(attribute);
+        if (count < minSupportMutable.longValue())
+          return frequentPatterns;
+      }
+    }
+    if (tree.singlePath()) {
+      return generateSinglePathPatterns(tree, K, minSupportMutable);
+    }
+    for (int i = tree.getHeaderTableCount() - 1; i >= 0; i--) {
+      int attribute = tree.getAttributeAtIndex(i);
+      long count = tree.getHeaderSupportCount(attribute);
+      if (count < minSupportMutable.longValue())
+        continue;
+      FPTree conditionalTree = treeCache.getTree(level);
+
+      if (conditionalOfCurrentAttribute == true) {
+        traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+            minSupportMutable, conditionalTree, tree);
+        returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+            K, treeCache, level + 1, true, currentAttribute);
+
+        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+            attribute, count, true, false);
+      } else {
+        if (attribute == currentAttribute) {
+          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+              minSupportMutable, conditionalTree, tree);
+          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+              K, treeCache, level + 1, true, currentAttribute);
+
+          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+              attribute, count, true, false);
+        } else if (attribute > currentAttribute) {
+          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+              minSupportMutable, conditionalTree, tree);
+          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+              K, treeCache, level + 1, false, currentAttribute);
+          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+              attribute, count, false, false);
+        }
+      }
+
+      if (frequentPatterns.isFull()) {
+        if (minSupportMutable.intValue() < frequentPatterns.leastSupport()) {
+          minSupportMutable.setValue(frequentPatterns.leastSupport());
+        }
+      }
+    }
+
+    return frequentPatterns;
+  }
+
+  /**
+   * Internal TopKFrequentPattern Generation algorithm, which represents the
+   * AttributePrimitives as integers and transforms features to use only
+   * integers
+   * 
+   * @param transactions Transaction database Iterator
+   * @param attributeFrequency array representing the Frequency of the
+   *        corresponding attribute id
+   * @param minSupport minimum support of the pattern to be mined
+   * @param K Max value of the Size of the Max-Heap in which Patterns are held
+   * @param featureSetSize number of features
+   * @param returnFeatures the id's of the features for which Top K patterns
+   *        have to be mined
+   * @param topKPatternsOutputCollector the outputCollector which transforms the
+   *        given Pattern in integer format to the corresponding
+   *        AttributePrimitive Format
+   * @return Top K frequent patterns for each attribute
+   * @throws IOException
+   */
+  final private Map<Integer, FrequentPatternMaxHeap> generateTopKFrequentPatterns(
+      Iterator<int[]> transactions,
+      long[] attributeFrequency,
+      long minSupport,
+      int K,
+      int featureSetSize,
+      Set<Integer> returnFeatures,
+      TopKPatternsOutputConvertor<AttributePrimitive> topKPatternsOutputCollector)
+      throws IOException {
+
+    
+    
+    FPTree tree = new FPTree(featureSetSize);
+    for (int i = 0; i < featureSetSize; i++) {
+      tree.addHeaderCount(i, attributeFrequency[i]);
+    }
+
+    //Constructing initial FPTree from the list of transactions
+    MutableLong minSupportMutable = new MutableLong(minSupport);
+    int nodecount = 0;
+    int attribcount = 0;
+    int i = 0;
+    while (transactions.hasNext()) {
+      int[] transaction = transactions.next();
+      Arrays.sort(transaction);
+      attribcount += transaction.length;
+      nodecount += treeAddCount(tree, transaction, 1, minSupportMutable,
+          attributeFrequency);
+      i++;
+      if (i % 10000 == 0) {
+        log.info("FPTree Building: Read {} Transactions", i);
+      }
+    }
+
+    log.info("Number of Nodes in the FP Tree: {}", nodecount);
+
+    return fpGrowth(tree, minSupportMutable, K, returnFeatures,
+        topKPatternsOutputCollector);
+  }
+
+  /**
+   * Generate the Feature Frequency list from the given transaction whose
+   * frequency > minSupport
+   * 
+   * @param transactions Iterator over the transaction database
+   * @param minSupport minSupport of the feature to be included
+   * @return the List of features and their associated frequency as a Pair
+   * @throws IOException
+   */
+  final public List<Pair<AttributePrimitive, Long>> generateFList(
+      Iterator<List<AttributePrimitive>> transactions, int minSupport)
+      throws IOException {
+
+    final Map<AttributePrimitive, MutableLong> AttributeSupport = new HashMap<AttributePrimitive, MutableLong>();
+    int count = 0;
+    while (transactions.hasNext()) {
+      List<AttributePrimitive> transaction = transactions.next();
+      for (AttributePrimitive attribute : transaction) {
+        if (AttributeSupport.containsKey(attribute) == false)
+          AttributeSupport.put(attribute, new MutableLong(1));
+        else
+          AttributeSupport.get(attribute).increment();
+        count++;
+      }
+    }
+    List<Pair<AttributePrimitive, Long>> fList = new ArrayList<Pair<AttributePrimitive, Long>>();
+    for (Entry<AttributePrimitive, MutableLong> e : AttributeSupport.entrySet())
+      fList.add(new Pair<AttributePrimitive, Long>(e.getKey(), e.getValue()
+          .longValue()));
+    Collections.sort(fList, new Comparator<Pair<AttributePrimitive, Long>>() {
+
+      @Override
+      public int compare(Pair<AttributePrimitive, Long> o1,
+          Pair<AttributePrimitive, Long> o2) {
+        int ret = o2.getSecond().compareTo(o1.getSecond());
+        if (ret != 0)
+          return ret;
+        return o1.getFirst().compareTo(o2.getFirst());
+      }
+
+    });
+
+    return fList;
+  }
+
+  /**
+   * Generate Top K Frequent Patterns for every feature in returnableFeatures
+   * given a stream of transactions and the minimum support
+   * 
+   * @param transactionStream Iterator of transaction
+   * @param frequencyList list of frequent features and their support value
+   * @param minSupport minimum support of the transactions
+   * @param K Number of top frequent patterns to keep
+   * @param returnableFeatures set of features for which the frequent patterns
+   *        are mined. If the set is null, then top K patterns for every
+   *        frequent item (an item whose support> minSupport) is generated
+   * @param output The output collector to which the the generated patterns are
+   *        written
+   * @throws IOException
+   */
+  final public void generateTopKFrequentPatterns(
+      Iterator<List<AttributePrimitive>> transactionStream,
+      List<Pair<AttributePrimitive, Long>> frequencyList,
+      long minSupport,
+      int K,
+      Set<AttributePrimitive> returnableFeatures,
+      OutputCollector<AttributePrimitive, List<Pair<List<AttributePrimitive>, Long>>> output)
+      throws IOException {
+
+    Map<Integer, AttributePrimitive> reverseMapping = new HashMap<Integer, AttributePrimitive>();
+    Map<AttributePrimitive, Integer> attributeIdMapping = new HashMap<AttributePrimitive, Integer>();
+
+    int id = 0;
+    for (Pair<AttributePrimitive, Long> feature : frequencyList) {
+      AttributePrimitive attrib = feature.getFirst();
+      Long frequency = feature.getSecond();
+      if (frequency.longValue() < minSupport)
+        continue;
+      attributeIdMapping.put(attrib, Integer.valueOf(id));
+      reverseMapping.put(Integer.valueOf(id++), attrib);
+    }
+
+    final long[] attributeFrequency = new long[attributeIdMapping.size()];
+    for (Pair<AttributePrimitive, Long> feature : frequencyList) {
+      AttributePrimitive attrib = feature.getFirst();
+      Long frequency = feature.getSecond();
+      if (frequency.longValue() < minSupport)
+        break;
+      attributeFrequency[attributeIdMapping.get(attrib)] = frequency
+          .longValue();
+    }
+
+    log.info("Number of unique items {}", frequencyList.size());
+
+    Set<Integer> returnFeatures = new HashSet<Integer>();
+    if (returnableFeatures.isEmpty() == false) {
+      for (AttributePrimitive attrib : returnableFeatures) {
+        if (attributeIdMapping.containsKey(attrib)) {
+          returnFeatures.add(attributeIdMapping.get(attrib));
+          log.info("Adding Pattern {}=>{}", attrib, attributeIdMapping
+              .get(attrib));
+        }
+      }
+    } else {
+      for (int j = 0; j < attributeIdMapping.size(); j++)
+        returnFeatures.add(j);
+    }
+
+    log.info("Number of unique pruned items {}", attributeIdMapping.size());
+    generateTopKFrequentPatterns(new TransactionIterator<AttributePrimitive>(
+        transactionStream, attributeIdMapping), attributeFrequency, minSupport,
+        K, reverseMapping.size(), returnFeatures,
+        new TopKPatternsOutputConvertor<AttributePrimitive>(output,
+            reverseMapping));
+
+  }
+
+  public static List<Pair<String, TopKStringPatterns>> readFrequentPattern(
+      FileSystem fs, Configuration conf, Path path) throws IOException {
+
+    List<Pair<String, TopKStringPatterns>> ret = new ArrayList<Pair<String, TopKStringPatterns>>();
+    Text key = new Text();
+    TopKStringPatterns value = new TopKStringPatterns();
+    SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, conf);
+    // key is feature value is count
+    while (reader.next(key, value)) {
+      ret.add(new Pair<String, TopKStringPatterns>(key.toString(),
+          new TopKStringPatterns(value.getPatterns())));
+    }
+    return ret;
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTree.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTree.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTree.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTree.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,396 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth.fpgrowth;
+
+import java.util.Arrays;
+
+public class FPTree {
+
+  final public static int DEFAULT_CHILDREN_INITIAL_SIZE = 2;
+
+  final public static int DEFAULT_HEADER_TABLE_INITIAL_SIZE = 4;
+
+  final public static int DEFAULT_INITIAL_SIZE = 8;
+
+  final public float GROWTH_RATE = 1.5f;
+
+  final public static int ROOTNODEID = 0;
+
+  final private static int HEADERTABLEBLOCKSIZE = 2;
+
+  final private static int HT_LAST = 1;
+
+  final private static int HT_NEXT = 0;
+
+  private int[] attribute;
+
+  private int[] childCount;
+
+  private int[] conditional;
+
+  private int[] headerTableAttributes;
+
+  private long[] headerTableAttributeCount;
+
+  private int headerTableCount = 0;
+
+  private int[] headerTableLookup;
+
+  private int[][] headerTableProperties;
+
+  private int[] next;
+
+  private int[][] nodeChildren;
+
+  private long[] nodeCount;
+
+  private int nodes = 0;
+
+  private int[] parent;
+
+  private boolean singlePath;
+
+  public FPTree() {
+    this(DEFAULT_INITIAL_SIZE, DEFAULT_HEADER_TABLE_INITIAL_SIZE);
+  }
+
+  public FPTree(int size) {
+    this(size, DEFAULT_HEADER_TABLE_INITIAL_SIZE);
+  }
+
+  public FPTree(int size, int headersize) {
+    if (size < DEFAULT_INITIAL_SIZE)
+      size = DEFAULT_INITIAL_SIZE;
+
+    parent = new int[size];
+    next = new int[size];
+    childCount = new int[size];
+    attribute = new int[size];
+    nodeCount = new long[size];
+
+    nodeChildren = new int[size][];
+    conditional = new int[size];
+
+    headerTableAttributes = new int[DEFAULT_HEADER_TABLE_INITIAL_SIZE];
+    headerTableAttributeCount = new long[DEFAULT_HEADER_TABLE_INITIAL_SIZE];
+    headerTableLookup = new int[DEFAULT_HEADER_TABLE_INITIAL_SIZE];
+    Arrays.fill(headerTableLookup, -1);
+    headerTableProperties = new int[DEFAULT_HEADER_TABLE_INITIAL_SIZE][];
+
+    singlePath = true;
+    createRootNode();
+  }
+
+  final public void addChild(int parentNodeId, int childnodeId) {
+    int length = childCount[parentNodeId];
+    if (length >= nodeChildren[parentNodeId].length) {
+      resizeChildren(parentNodeId);
+    }
+    nodeChildren[parentNodeId][length++] = childnodeId;
+    childCount[parentNodeId] = length;
+
+    if (length > 1 && singlePath) {
+      singlePath = false;
+    }
+  }
+
+  final public boolean addCount(int nodeId, long nextNodeCount) {
+    if (nodeId < nodes) {
+      this.nodeCount[nodeId] += nextNodeCount;
+      return true;
+    }
+    return false;
+  }
+
+  final public void addHeaderCount(int attribute, long count) {
+    int index = getHeaderIndex(attribute);
+    headerTableAttributeCount[index] += count;
+  }
+
+  final public void addHeaderNext(int attribute, int nodeId) {
+    int index = getHeaderIndex(attribute);
+    if (headerTableProperties[index][HT_NEXT] == -1) {
+      headerTableProperties[index][HT_NEXT] = nodeId;
+      headerTableProperties[index][HT_LAST] = nodeId;
+    } else {
+      setNext(headerTableProperties[index][HT_LAST], nodeId);
+      headerTableProperties[index][HT_LAST] = nodeId;
+    }
+  }
+
+  final public int attribute(int nodeId) {
+    return this.attribute[nodeId];
+  }
+
+  final public int childAtIndex(int nodeId, int index) {
+    if (childCount[nodeId] < index) {
+      return -1;
+    }
+    return nodeChildren[nodeId][index];
+  }
+
+  final public int childCount(int nodeId) {
+    return childCount[nodeId];
+  }
+
+  final public int childWithAttribute(int nodeId, int childAttribute) {
+    int length = childCount[nodeId];
+    for (int i = 0; i < length; i++) {
+      if (attribute[nodeChildren[nodeId][i]] == childAttribute)
+        return nodeChildren[nodeId][i];
+    }
+    return -1;
+  }
+
+  final public void clear() {
+    nodes = 0;
+    headerTableCount = 0;
+    Arrays.fill(headerTableLookup, -1);
+    createRootNode();
+  }
+
+  final public void clearConditional() {
+    for (int i = nodes - 1; i >= 0; i--)
+      conditional[i] = 0;
+  }
+
+  final public int conditional(int nodeId) {
+    return this.conditional[nodeId];
+  }
+
+  final public long count(int nodeId) {
+    return nodeCount[nodeId];
+  }
+
+  final public int createConditionalNode(int attribute, long count) {
+    if (nodes >= this.attribute.length) {
+      resize();
+    }
+    childCount[nodes] = 0;
+    next[nodes] = -1;
+    parent[nodes] = -1;
+    conditional[nodes] = 0;
+    this.attribute[nodes] = attribute;
+    nodeCount[nodes] = count;
+
+    if (nodeChildren[nodes] == null)
+      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+
+    int childNodeId = nodes++;
+    return childNodeId;
+  }
+
+  final public int createNode(int parentNodeId, int attribute, long count) {
+    if (nodes >= this.attribute.length) {
+      resize();
+    }
+
+    childCount[nodes] = 0;
+    next[nodes] = -1;
+    parent[nodes] = parentNodeId;
+    this.attribute[nodes] = attribute;
+    nodeCount[nodes] = count;
+
+    conditional[nodes] = 0;
+    if (nodeChildren[nodes] == null)
+      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+
+    int childNodeId = nodes++;
+    addChild(parentNodeId, childNodeId);
+    addHeaderNext(attribute, childNodeId);
+    return childNodeId;
+  }
+
+  final public int createRootNode() {
+    childCount[nodes] = 0;
+    next[nodes] = -1;
+    parent[nodes] = 0;
+    attribute[nodes] = -1;
+    nodeCount[nodes] = 0;
+    if (nodeChildren[nodes] == null)
+      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+    int childNodeId = nodes++;
+    return childNodeId;
+  }
+
+  final public int getAttributeAtIndex(int index) {
+    return headerTableAttributes[index];
+  }
+
+  final public int getHeaderNext(int attribute) {
+    int index = getHeaderIndex(attribute);
+    return headerTableProperties[index][HT_NEXT];
+  }
+
+  final public long getHeaderSupportCount(int attribute) {
+    int index = getHeaderIndex(attribute);
+    return headerTableAttributeCount[index];
+  }
+
+  final public int[] getHeaderTableAttributes() {
+    int[] attributes = new int[headerTableCount];
+    System.arraycopy(headerTableAttributes, 0, attributes, 0, headerTableCount);
+    return attributes;
+  }
+
+  final public int getHeaderTableCount() {
+    return headerTableCount;
+  }
+
+  final public int next(int nodeId) {
+    return next[nodeId];
+  }
+
+  final public boolean isEmpty() {
+    return nodes <= 1;
+  }
+
+  final public int parent(int nodeId) {
+    return parent[nodeId];
+  }
+
+  final public void reorderHeaderTable() {
+    Arrays.sort(headerTableAttributes, 0, headerTableCount);
+  }
+
+  final public boolean setConditional(int nodeId, int conditional) {
+    if (nodeId < nodes) {
+      this.conditional[nodeId] = conditional;
+      return true;
+    }
+    return false;
+  }
+
+  final public boolean setNext(int nodeId, int next) {
+    if (nodeId < nodes) {
+      this.next[nodeId] = next;
+      return true;
+    }
+    return false;
+  }
+
+  final public boolean setParent(int nodeId, int parent) {
+    if (nodeId < nodes) {
+      this.parent[nodeId] = parent;
+
+      int length = childCount[parent];
+      if (length >= nodeChildren[parent].length) {
+        resizeChildren(parent);
+      }
+      nodeChildren[parent][length++] = nodeId;
+      childCount[parent] = length;
+      return true;
+    }
+    return false;
+  }
+
+  final public void setSinglePath(boolean bit) {
+    singlePath = bit;
+  }
+
+  final public boolean singlePath() {
+    return singlePath;
+  }
+
+  final private int getHeaderIndex(int attribute) {
+    if (attribute >= headerTableLookup.length)
+      resizeHeaderLookup(attribute);
+    int index = headerTableLookup[attribute];
+    if (index == -1) { // if attribute didnt exist;
+      if (headerTableCount >= headerTableAttributes.length)
+        resizeHeaderTable();
+      headerTableAttributes[headerTableCount] = attribute;
+      if (headerTableProperties[headerTableCount] == null)
+        headerTableProperties[headerTableCount] = new int[HEADERTABLEBLOCKSIZE];
+      headerTableAttributeCount[headerTableCount] = 0;
+      headerTableProperties[headerTableCount][HT_NEXT] = -1;
+      headerTableProperties[headerTableCount][HT_LAST] = -1;
+      index = headerTableCount++;
+      headerTableLookup[attribute] = index;
+    }
+    return index;
+  }
+
+  final private void resize() {
+    int size = (int) (GROWTH_RATE * nodes);
+    if (size < DEFAULT_INITIAL_SIZE)
+      size = DEFAULT_INITIAL_SIZE;
+
+    int[] oldChildCount = childCount;
+    int[] oldAttribute = attribute;
+    long[] oldnodeCount = nodeCount;
+    int[] oldParent = parent;
+    int[] oldNext = next;
+    int[][] oldNodeChildren = nodeChildren;
+    int[] oldConditional = conditional;
+
+    childCount = new int[size];
+    attribute = new int[size];
+    nodeCount = new long[size];
+    parent = new int[size];
+    next = new int[size];
+
+    nodeChildren = new int[size][];
+    conditional = new int[size];
+
+    System.arraycopy(oldChildCount, 0, this.childCount, 0, nodes);
+    System.arraycopy(oldAttribute, 0, this.attribute, 0, nodes);
+    System.arraycopy(oldnodeCount, 0, this.nodeCount, 0, nodes);
+    System.arraycopy(oldParent, 0, this.parent, 0, nodes);
+    System.arraycopy(oldNext, 0, this.next, 0, nodes);
+    System.arraycopy(oldNodeChildren, 0, this.nodeChildren, 0, nodes);
+    System.arraycopy(oldConditional, 0, this.conditional, 0, nodes);
+  }
+
+  final private void resizeChildren(int nodeId) {
+    int length = childCount[nodeId];
+    int size = (int) (GROWTH_RATE * (length));
+    if (size < DEFAULT_CHILDREN_INITIAL_SIZE)
+      size = DEFAULT_CHILDREN_INITIAL_SIZE;
+    int[] oldNodeChildren = nodeChildren[nodeId];
+    nodeChildren[nodeId] = new int[size];
+    System.arraycopy(oldNodeChildren, 0, this.nodeChildren[nodeId], 0, length);
+  }
+
+  final private void resizeHeaderLookup(int attribute) {
+    int size = (int) (attribute * GROWTH_RATE);
+    int[] oldLookup = headerTableLookup;
+    headerTableLookup = new int[size];
+    Arrays.fill(headerTableLookup, oldLookup.length, size, -1);
+    System.arraycopy(oldLookup, 0, this.headerTableLookup, 0, oldLookup.length);
+  }
+
+  final private void resizeHeaderTable() {
+    int size = (int) (GROWTH_RATE * (headerTableCount));
+    if (size < DEFAULT_HEADER_TABLE_INITIAL_SIZE)
+      size = DEFAULT_HEADER_TABLE_INITIAL_SIZE;
+
+    int[] oldAttributes = headerTableAttributes;
+    long[] oldAttributeCount = headerTableAttributeCount;
+    int[][] oldProperties = headerTableProperties;
+    headerTableAttributes = new int[size];
+    headerTableAttributeCount = new long[size];
+    headerTableProperties = new int[size][];
+    System.arraycopy(oldAttributes, 0, this.headerTableAttributes, 0,
+        headerTableCount);
+    System.arraycopy(oldAttributeCount, 0, this.headerTableAttributeCount, 0,
+        headerTableCount);
+    System.arraycopy(oldProperties, 0, this.headerTableProperties, 0,
+        headerTableCount);
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTreeDepthCache.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTreeDepthCache.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTreeDepthCache.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTreeDepthCache.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth.fpgrowth;
+
+import java.util.ArrayList;
+
+import org.apache.mahout.common.cache.LeastKCache;
+
+public class FPTreeDepthCache {
+
+  public static int FirstLevelCacheSize = 5;
+
+  final private ArrayList<FPTree> treeCache = new ArrayList<FPTree>();
+
+  final private LeastKCache<Integer, FPTree> firstLevelCache = new LeastKCache<Integer, FPTree>(
+      FirstLevelCacheSize);
+
+  final public FPTree getTree(int level) {
+    while (treeCache.size() < level + 1) {
+      FPTree cTree = new FPTree();
+      treeCache.add(cTree);
+    }
+    FPTree conditionalTree = treeCache.get(level);
+    conditionalTree.clear();
+    return conditionalTree;
+  }
+
+  private int hits = 0;
+
+  private int misses = 0;
+
+  final public FPTree getFirstLevelTree(int attr) {
+    Integer attribute = Integer.valueOf(attr);
+    if (firstLevelCache.contains(attribute)) {
+      hits++;
+      return firstLevelCache.get(attribute);
+    } else {
+      misses++;
+      FPTree conditionalTree = new FPTree();
+      firstLevelCache.set(attribute, conditionalTree);
+      return conditionalTree;
+    }
+  }
+
+  final public int getHits() {
+    return hits;
+  }
+
+  final public int getMisses() {
+    return misses;
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FrequentPatternMaxHeap.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth.fpgrowth;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeSet;
+
+/**
+ * {@link FrequentPatternMaxHeap} keeps top K Attributes in a TreeSet
+ * 
+ * @param <T>
+ */
+public class FrequentPatternMaxHeap {
+  TreeSet<Pattern> set = null;
+
+  HashMap<Long, Set<Pattern>> patternIndex = null;
+
+  Pattern least = null;
+
+  int maxSize = 0;
+
+  int count = 0;
+
+  private Comparator<Pattern> treeSetComparator = new Comparator<Pattern>() {
+    @Override
+    final public int compare(Pattern cr1, Pattern cr2) {
+      long support2 = cr2.getSupport();
+      long support1 = cr1.getSupport();
+      int length2 = cr2.length();
+      int length1 = cr1.length();
+      if (support1 == support2) {
+        if (length1 == length2) {//if they are of same length and support order randomly
+          return 1;
+        } else
+          return length2 - length1;
+      } else {
+        if( support2 - support1 >0)
+          return 1;
+        else return -1;
+      }
+    }
+  };
+
+  public FrequentPatternMaxHeap(int numResults) {
+    maxSize = numResults;
+    set = new TreeSet<Pattern>(treeSetComparator);
+  }
+
+  final public void insert(Pattern frequentPattern) {
+    insert(frequentPattern, true);
+  }
+
+  final public void insert(Pattern frequentPattern, boolean subPatternCheck) {
+    if (subPatternCheck)// lazy initialization
+    {
+      if (patternIndex == null)
+        patternIndex = new HashMap<Long, Set<Pattern>>();
+    }
+    if (count == maxSize) {
+      int cmp = treeSetComparator.compare(frequentPattern, least);
+      if (cmp < 0) {
+        if (addPattern(frequentPattern, subPatternCheck)) {
+          Pattern evictedItem = set.pollLast();
+          least = set.last();
+          if (subPatternCheck)
+            patternIndex.get(evictedItem.support).remove(evictedItem);
+        }
+      }
+    } else {           
+      if (addPattern(frequentPattern, subPatternCheck)){
+        count++;        
+        if(least != null) {
+          int cmp = treeSetComparator.compare(least, frequentPattern);   
+          if (cmp < 0)
+            least = frequentPattern;
+          else if(cmp == 0)
+            return;
+        }
+        else if (least == null)
+          least = frequentPattern;
+      }
+    }
+  }
+
+  final private boolean addPattern(Pattern frequentPattern,
+      boolean subPatternCheck) {
+    if (subPatternCheck == false) {
+      set.add(frequentPattern);
+      return true;
+    } else {
+      Long index = Long.valueOf(frequentPattern.support);
+      if (patternIndex.containsKey(index)) {
+        Set<Pattern> indexSet = patternIndex.get(index);
+        boolean replace = false;
+        Pattern replacablePattern = null;
+        for (Pattern p : indexSet) {
+
+          if (frequentPattern.isSubPatternOf(p))
+            return false;
+          else if (p.isSubPatternOf(frequentPattern)) {
+            replace = true;
+            replacablePattern = p;
+            break;
+          }
+        }
+        if (replace) {
+          indexSet.remove(replacablePattern);
+          if (set.remove(replacablePattern))
+            count--;
+          if (indexSet.contains(frequentPattern) == false) {
+            if (set.add(frequentPattern))
+              count++;
+            indexSet.add(frequentPattern);
+          }
+          return false;
+        }
+        set.add(frequentPattern);
+        indexSet.add(frequentPattern);
+        return true;
+      } else {
+        set.add(frequentPattern);
+        Set<Pattern> patternList = null;
+        if (patternIndex.containsKey(index) == false) {
+          patternList = new HashSet<Pattern>();
+          patternIndex.put(index, patternList);
+        }
+        patternList = patternIndex.get(index);
+        patternList.add(frequentPattern);
+
+        return true;
+      }
+    }
+  }
+
+  final public TreeSet<Pattern> getHeap() {
+    return set;
+  }
+
+  final public boolean addable(long support) {
+    if (count < maxSize)
+      return true;
+    if (least.support > support)
+      return false;
+    return true;
+  }
+
+  @Override
+  final public String toString() {
+    return super.toString();
+  }
+
+  final public boolean isFull() {
+    return count == maxSize;
+  }
+
+  final public long leastSupport() {
+    if (least == null)
+      return 0;
+    return least.support;
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/Pattern.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/Pattern.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/Pattern.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/Pattern.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth.fpgrowth;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class Pattern implements Serializable, Cloneable {
+
+  private static final long serialVersionUID = 8698199782842762173L;
+
+  private int hashCode;
+
+  private boolean dirty = true;
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    if (dirty == false)
+      return hashCode;
+    int result = 1;
+    result = prime * result + Arrays.hashCode(pattern);
+    result = prime * result + Long.valueOf(support).hashCode();
+    hashCode = result;
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    Pattern other = (Pattern) obj;
+    if (length != other.length)
+      return false;
+    if (support != other.support)
+      return false;
+    if (!Arrays.equals(pattern, other.pattern))
+      return false;
+    return true;
+  }
+
+  public static int DEFAULT_INITIAL_SIZE = 2;
+
+  public static float GROWTH_RATE = 1.5f;
+
+  int[] pattern;
+
+  long[] supportValues;
+
+  int length = 0;
+
+  long support = Long.MAX_VALUE;
+
+  public long getSupport() {
+    return this.support;
+  }
+
+  final public Object[] getPatternWithSupport() {
+    return new Object[] { this.pattern, this.supportValues };
+  }
+
+  final public int[] getPattern() {
+    return this.pattern;
+  }
+
+  final public int length() {
+    return this.length;
+  }
+
+  public Pattern() {
+    this(DEFAULT_INITIAL_SIZE);
+  }
+
+  public Pattern(int size) {
+    if (size < DEFAULT_INITIAL_SIZE)
+      size = DEFAULT_INITIAL_SIZE;
+    this.pattern = new int[size];
+    this.supportValues = new long[size];
+    dirty = true;
+  }
+
+  final public void add(int id, long support) {
+    if (length >= pattern.length)
+      resize();
+    this.pattern[length] = id;
+    this.supportValues[length++] = support;
+    this.support = (support > this.support) ? this.support : support;
+    dirty = true;
+  }
+
+  final private void resize() {
+    int size = (int) (GROWTH_RATE * length);
+    if (size < DEFAULT_INITIAL_SIZE)
+      size = DEFAULT_INITIAL_SIZE;
+    int[] oldpattern = pattern;
+    long[] oldSupport = supportValues;
+    this.pattern = new int[size];
+    this.supportValues = new long[size];
+    System.arraycopy(oldpattern, 0, this.pattern, 0, length);
+    System.arraycopy(oldSupport, 0, this.supportValues, 0, length);
+  }
+
+  @Override
+  final public String toString() {
+    int[] arr = new int[length];
+    System.arraycopy(pattern, 0, arr, 0, length);
+    return Arrays.toString(arr) + "-" + support;
+  }
+
+  final public boolean isSubPatternOf(Pattern frequentPattern) {// Patterns are
+    // in the sorted
+    // order of
+    // their ids
+    int[] otherPattern = frequentPattern.getPattern();
+    int otherLength = frequentPattern.length();
+    int otherI = 0;
+    int i = 0;
+    if (this.length() > frequentPattern.length())
+      return false;
+    while (i < length && otherI < otherLength) {
+      if (otherPattern[otherI] == pattern[i]) {
+        otherI++;
+        i++;
+        continue;
+      } else if (otherPattern[otherI] < pattern[i]) {
+        otherI++;
+      } else
+        return false;
+    }
+    if (otherI == otherLength && i != length)
+      return false;
+    return true;
+
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/package.html
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/package.html?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/package.html (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/package.html Sun Oct 18 22:19:56 2009
@@ -0,0 +1,69 @@
+<HTML>
+ <!--
+ /**
+ * 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.
+ */
+
+ -->
+ <HEAD>
+    <TITLE>org.apache.mahout.fpm.pfpgrowth</TITLE>
+</HEAD>
+<BODY>
+<DIV align="center">
+Map/Reduce(Parallel) implementation of <b>FP Growth</b> Algorithm for frequent Itemset Mining
+</DIV>
+<DIV>&nbsp;</DIV>
+<DIV>
+We have a Top K Parallel FPGrowth Implementation. What it means is that given a huge transaction list, we find all unique features(field values) 
+ and eliminates those features whose frequency in the whole dataset is less that <i>minSupport</i>. Using these remaining 
+ features N, we find the top K closed patterns for each of them, generating NK patterns. <i>FPGrowth</i> Algorithm is a generic implementation, we can use 
+ any Object type to denote a feature. Current implementation requires you to use a String as the object type. You may implement a version for any object 
+ by creating Iterators, Convertors and TopKPatternWritable for that particular object. For more information please refer the package 
+ <i> org.apache.mahout.fpm.pfpgrowth.convertors.string </i> 
+ <br>
+ e.g:
+ <pre>
+ FPGrowth<String> fp = new FPGrowth<String>();
+ Set<String> features = new HashSet<String>();
+ fp.generateTopKStringFrequentPatterns(
+ 		new StringRecordIterator(new FileLineIterable(new File(input), encoding, false), pattern), 
+        fp.generateFList(
+        	new StringRecordIterator(new FileLineIterable(new File(input), encoding, false), pattern), minSupport),
+       	minSupport,
+        maxHeapSize, 
+        features,
+        new StringOutputConvertor(new SequenceFileOutputCollector<Text, TopKStringPatterns>(writer))
+  );
+ </pre>
+ <ul>
+<li>The first argument is the iterator of transaction in this case its Iterator&lt;List&lt;String&gt;&gt;
+</li><li>The second argument is the output of generateFList function, which returns the frequent items and their frequencies from the given database transaction iterator 
+</li><li>The third argument is the minimum Support of the pattern to be generated
+</li><li>The fourth argument is the maximum number of patterns to be mined for each feature 
+</li><li>The fifth argument is the set of features for which the frequent patterns has to be mined
+</li><li>The last argument is an output collector which takes [key, value] of Feature and TopK Patterns of the format [String, List&lt;Pair&lt;List&lt;String&gt;, Long&gt;&gt;] and writes them to the appropriate writer class which takes care of storing the object, in this case in a Sequence File Output format
+</li></ul>
+
+<p>The command line launcher for string transaction data <i>org.apache.mahout.fpm.pfpgrowth.FPGrowthJob</i> has other features including specifying the regex pattern for spitting a string line of a transaction into the constituent features</p>
+The <i>numGroups</i> parameter in FPGrowthJob specifies the number of groups into which transactions have to be decomposed. 
+The <i>numTreeCacheEntries</i> parameter specifies the number of generated conditional FP-Trees to be kept in memory so as not to regenerate them. Increasing this number increases the memory consumption but might improve speed until a certain point. This depends entirely on the dataset in question. A value of 5-10 is recommended for mining upto top 100 patterns for each feature 
+</DIV>
+<DIV>&nbsp;</DIV>
+<DIV align="center">
+Copyright &copy; 2009 <A HREF="http://lucene.apache.org/mahout">Apache Software Foundation - Mahout</A>
+</DIV>
+</BODY>
+</HTML>

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthTest.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.mahout.common.Pair;
+import org.apache.mahout.fpm.pfpgrowth.convertors.SequenceFileOutputCollector;
+import org.apache.mahout.fpm.pfpgrowth.convertors.string.StringOutputConvertor;
+import org.apache.mahout.fpm.pfpgrowth.convertors.string.TopKStringPatterns;
+import org.apache.mahout.fpm.pfpgrowth.fpgrowth.FPGrowth;
+
+import junit.framework.TestCase;
+
+public class FPGrowthTest extends TestCase {
+
+  public FPGrowthTest(String s) {
+    super(s);
+  }
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+
+  }
+
+  public void testMaxHeapFPGrowth() throws IOException {
+
+    FPGrowth<String> fp = new FPGrowth<String>();
+
+    Collection<List<String>> transactions = new ArrayList<List<String>>();
+    transactions.add(Arrays.asList("E", "A", "D", "B"));
+    transactions.add(Arrays.asList("D", "A", "C", "E", "B"));
+    transactions.add(Arrays.asList("C", "A", "B", "E"));
+    transactions.add(Arrays.asList("B", "A", "D"));
+    transactions.add(Arrays.asList("D"));
+    transactions.add(Arrays.asList("D", "B"));
+    transactions.add(Arrays.asList("A", "D", "E"));
+    transactions.add(Arrays.asList("B", "C"));
+
+    File tmpDir = new File(System.getProperty("java.io.tmpdir"));
+    File tmpLoc = new File(tmpDir, "fpgrowthTest");
+    tmpLoc.mkdirs();
+    File tmpFile = File.createTempFile("fpgrowthTest", ".dat", tmpLoc);
+
+    Path path = new Path(tmpFile.getAbsolutePath());
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.get(conf);
+
+    SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, path,
+        Text.class, TopKStringPatterns.class);
+    fp.generateTopKFrequentPatterns(transactions.iterator(), fp
+        .generateFList(transactions.iterator(), 3), 3, 100,
+        new HashSet<String>(), new StringOutputConvertor(
+            new SequenceFileOutputCollector<Text, TopKStringPatterns>(writer)
+            ));
+    writer.close();
+
+    List<Pair<String, TopKStringPatterns>> frequentPatterns = FPGrowth
+        .readFrequentPattern(fs, conf, path);
+    assertEquals(
+        frequentPatterns.toString()
+            + " is not equal to [(C,([B, C],3)), (E,([A, E],4), ([D, A, E],3), ([B, A, E],3)), (A,([A],5), ([B, A],4), ([D, A],4), ([A, E],4), ([B, D, A],3), ([D, A, E],3), ([B, A, E],3)), (D,([D],6), ([B, D],4), ([D, A],4), ([B, D, A],3), ([D, A, E],3)), (B,([B],6), ([B, D],4), ([B, A],4), ([B, D, A],3), ([B, A, E],3), ([B, C],3))]",
+        "[(C,([B, C],3)), (E,([A, E],4), ([D, A, E],3), ([B, A, E],3)), (A,([A],5), ([B, A],4), ([D, A],4), ([A, E],4), ([B, D, A],3), ([D, A, E],3), ([B, A, E],3)), (D,([D],6), ([B, D],4), ([D, A],4), ([B, D, A],3), ([D, A, E],3)), (B,([B],6), ([B, D],4), ([B, A],4), ([B, D, A],3), ([B, A, E],3), ([B, C],3))]",
+        frequentPatterns.toString());
+
+  }
+}

Added: lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java (added)
+++ lucene/mahout/trunk/core/src/test/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowthTest.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.mahout.common.Pair;
+import org.apache.mahout.common.Parameters;
+import org.apache.mahout.fpm.pfpgrowth.convertors.string.TopKStringPatterns;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import junit.framework.TestCase;
+
+public class PFPGrowthTest extends TestCase {
+
+  private static final Logger log = LoggerFactory
+      .getLogger(PFPGrowthTest.class);
+
+  protected Parameters params = new Parameters();
+
+  protected void setUp() throws Exception {
+    super.setUp();
+    params.set("minSupport", "3");
+    params.set("maxHeapSize", "4");
+    params.set("numGroups", "2");
+    params.set("encoding", "UTF-8");
+    params.set("input", "testdata/transactions");
+    params.set("output", "output/frequentpatterns");
+    File testData = new File("testdata");
+    if (!testData.exists()) {
+      testData.mkdir();
+    }
+    testData = new File("testdata/transactions");
+    if (!testData.exists()) {
+      testData.mkdir();
+    }
+    BufferedWriter writer = new BufferedWriter(new FileWriter(
+        "testdata/transactions/test.txt"));
+    Collection<List<String>> transactions = new ArrayList<List<String>>();
+    transactions.add(Arrays.asList("E", "A", "D", "B"));
+    transactions.add(Arrays.asList("D", "A", "C", "E", "B"));
+    transactions.add(Arrays.asList("C", "A", "B", "E"));
+    transactions.add(Arrays.asList("B", "A", "D"));
+    transactions.add(Arrays.asList("D"));
+    transactions.add(Arrays.asList("D", "B"));
+    transactions.add(Arrays.asList("A", "D", "E"));
+    transactions.add(Arrays.asList("B", "C"));
+    for (List<String> transaction : transactions) {
+      String sep = "";
+      for (String item : transaction) {
+        writer.write(sep + item);
+        sep = ",";
+      }
+      writer.write("\n");
+    }
+    writer.close();
+
+  }
+
+  public void testStartParallelCounting() throws IOException, InterruptedException, ClassNotFoundException {
+    log.info("Starting Parallel Counting Test: {}", params.get("maxHeapSize"));
+    PFPGrowth.startParallelCounting(params);
+    log.info("Reading fList Test: {}", params.get("maxHeapSize"));
+    List<Pair<String, Long>> fList = PFPGrowth.readFList(params);
+    log.info("{}", fList);
+    assertEquals("[(B,6), (D,6), (A,5), (E,4), (C,3)]", fList.toString());
+  }
+
+  public void testStartGroupingItems() throws IOException {
+    log.info("Starting Grouping Test: {}", params.get("maxHeapSize"));
+    PFPGrowth.startGroupingItems(params);
+    Map<String, Long> gList = PFPGrowth.deserializeMap(params, "gList",
+        new Configuration());
+    log.info("{}", gList);
+    assertEquals("{D=0, E=1, A=0, B=0, C=1}", gList.toString());
+  }
+
+  public void testStartParallelFPGrowth() throws IOException, InterruptedException, ClassNotFoundException {
+    log.info("Starting Parallel FPGrowth Test: {}", params.get("maxHeapSize"));
+    PFPGrowth.startGroupingItems(params);
+    PFPGrowth.startParallelFPGrowth(params);
+    log
+        .info("Starting Pattern Aggregation Test: {}", params
+            .get("maxHeapSize"));
+    PFPGrowth.startAggregating(params);
+    List<Pair<String, TopKStringPatterns>> frequentPatterns = PFPGrowth
+        .readFrequentPattern(params);
+    assertEquals(
+        "[(A,([A],5), ([D, A],4), ([B, A],4), ([A, E],4)), (B,([B],6), ([B, D],4), ([B, A],4), ([B, D, A],3)), (C,([B, C],3)), (D,([D],6), ([D, A],4), ([B, D],4), ([D, A, E],3)), (E,([A, E],4), ([D, A, E],3), ([B, A, E],3))]",
+        frequentPatterns.toString());
+
+  }
+
+}

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthJob.java?rev=826539&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthJob.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthJob.java Sun Oct 18 22:19:56 2009
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.fpm.pfpgrowth;
+
+import org.apache.commons.cli2.CommandLine;
+import org.apache.commons.cli2.Group;
+import org.apache.commons.cli2.Option;
+import org.apache.commons.cli2.OptionException;
+import org.apache.commons.cli2.builder.ArgumentBuilder;
+import org.apache.commons.cli2.builder.DefaultOptionBuilder;
+import org.apache.commons.cli2.builder.GroupBuilder;
+import org.apache.commons.cli2.commandline.Parser;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.mahout.common.CommandLineUtil;
+import org.apache.mahout.common.FileLineIterable;
+import org.apache.mahout.common.Pair;
+import org.apache.mahout.common.Parameters;
+import org.apache.mahout.common.StringRecordIterator;
+import org.apache.mahout.common.commandline.DefaultOptionCreator;
+import org.apache.mahout.fpm.pfpgrowth.convertors.SequenceFileOutputCollector;
+import org.apache.mahout.fpm.pfpgrowth.convertors.string.StringOutputConvertor;
+import org.apache.mahout.fpm.pfpgrowth.convertors.string.TopKStringPatterns;
+import org.apache.mahout.fpm.pfpgrowth.fpgrowth.FPGrowth;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class FPGrowthJob {
+
+  private static final Logger log = LoggerFactory.getLogger(FPGrowthJob.class);
+
+  /**
+   * Run TopK FPGrowth given the input file,
+   * 
+   * @param args
+   * @throws IOException
+   * @throws OptionException
+   * @throws NumberFormatException
+   * @throws IllegalStateException
+   * @throws InterruptedException
+   * @throws ClassNotFoundException
+   */
+  public static void main(String[] args) throws IOException, OptionException,
+      NumberFormatException, IllegalStateException, InterruptedException,
+      ClassNotFoundException {
+    DefaultOptionBuilder obuilder = new DefaultOptionBuilder();
+    ArgumentBuilder abuilder = new ArgumentBuilder();
+    GroupBuilder gbuilder = new GroupBuilder();
+
+    Option inputDirOpt = obuilder.withLongName("input").withRequired(true)
+        .withArgument(
+            abuilder.withName("input").withMinimum(1).withMaximum(1).create())
+        .withDescription(
+            "The Directory on HDFS containing the transaction files")
+        .withShortName("i").create();
+
+    Option outputOpt = DefaultOptionCreator.outputOption(obuilder, abuilder).create();
+
+    Option helpOpt = DefaultOptionCreator.helpOption(obuilder);
+
+    // minSupport(3), maxHeapSize(50), numGroups(1000)
+    Option minSupportOpt = obuilder.withLongName("minSupport").withArgument(
+        abuilder.withName("minSupport").withMinimum(1).withMaximum(1).create())
+        .withDescription("(Optional) Minimum Support. Default Value: 3")
+        .withShortName("s").create();
+
+    Option maxHeapSizeOpt = obuilder
+        .withLongName("maxHeapSize")
+        .withArgument(
+            abuilder.withName("maxHeapSize").withMinimum(1).withMaximum(1)
+                .create())
+        .withDescription(
+            "(Optional) Maximum Heap Size k, to denote the requirement to mine top K items. Default value: 50")
+        .withShortName("k").create();
+
+    Option numGroupsOpt = obuilder
+        .withLongName("numGroups")
+        .withArgument(
+            abuilder.withName("numGroups").withMinimum(1).withMaximum(1)
+                .create())
+        .withDescription(
+            "(Optional) Number of groups the features should be divided in the map-reduce version. Doesnt work in sequential verion Default Value:1000")
+        .withShortName("g").create();
+
+    Option recordSplitterOpt = obuilder
+        .withLongName("splitterPattern")
+        .withArgument(
+            abuilder.withName("splitterPattern").withMinimum(1).withMaximum(1)
+                .create())
+        .withDescription(
+            "Regular Expression pattern used to split given string transaction into itemsets. Default value splits comma separated itemsets.  Default Value: \"[ ,\\t]*[,|\\t][ ,\\t]*\" ")
+        .withShortName("regex").create();
+
+    Option treeCacheOpt = obuilder
+        .withLongName("numTreeCacheEntries")
+        .withArgument(
+            abuilder.withName("numTreeCacheEntries").withMinimum(1)
+                .withMaximum(1).create())
+        .withDescription(
+            "(Optional) Number of entries in the tree cache to prevent duplicate tree building. "
+                + "(Warning) a first level conditional FP-Tree might consume a lot of memory, "
+                + "so keep this value small, but big enough to prevent duplicate tree building. Default Value:5 Recommended Values: [5-10]")
+        .withShortName("tc").create();
+
+    Option methodOpt = obuilder.withLongName("method").withRequired(true)
+        .withArgument(
+            abuilder.withName("method").withMinimum(1).withMaximum(1).create())
+        .withDescription("Method of processing: sequential|mapreduce")
+        .withShortName("method").create();
+    Option encodingOpt = obuilder.withLongName("encoding").withArgument(
+        abuilder.withName("encoding").withMinimum(1).withMaximum(1).create())
+        .withDescription("(Optional) The file encoding.  Default value: UTF-8")
+        .withShortName("e").create();
+
+    Group group = gbuilder.withName("Options").withOption(minSupportOpt)
+        .withOption(inputDirOpt).withOption(outputOpt).withOption(
+            maxHeapSizeOpt).withOption(numGroupsOpt).withOption(methodOpt)
+        .withOption(encodingOpt).withOption(helpOpt).withOption(treeCacheOpt)
+        .withOption(recordSplitterOpt).create();
+
+    Parser parser = new Parser();
+    parser.setGroup(group);
+    CommandLine cmdLine = parser.parse(args);
+
+    if (cmdLine.hasOption(helpOpt)) {
+      CommandLineUtil.printHelp(group);
+      System.exit(0);
+    }
+
+    Parameters params = new Parameters();
+
+    if (cmdLine.hasOption(minSupportOpt)) {
+      String minSupportString = (String) cmdLine.getValue(minSupportOpt);
+      params.set("minSupport", minSupportString);
+    }
+    if (cmdLine.hasOption(maxHeapSizeOpt)) {
+      String maxHeapSizeString = (String) cmdLine.getValue(maxHeapSizeOpt);
+      params.set("maxHeapSize", maxHeapSizeString);
+    }
+    if (cmdLine.hasOption(numGroupsOpt)) {
+      String numGroupsString = (String) cmdLine.getValue(numGroupsOpt);
+      params.set("numGroups", numGroupsString);
+    }
+
+    if (cmdLine.hasOption(treeCacheOpt)) {
+      String numTreeCacheString = (String) cmdLine.getValue(treeCacheOpt);
+      params.set("treeCacheSize", numTreeCacheString);
+    }
+
+    if (cmdLine.hasOption(recordSplitterOpt)) {
+      String patternString = (String) cmdLine.getValue(recordSplitterOpt);
+      params.set("splitPattern", patternString);
+    }
+
+    String encoding = "UTF-8";
+    if (cmdLine.hasOption(encodingOpt)) {
+      encoding = (String) cmdLine.getValue(encodingOpt);
+    }
+    params.set("encoding", encoding);
+    String inputDir = (String) cmdLine.getValue(inputDirOpt);
+    String outputDir = (String) cmdLine.getValue(outputOpt);
+
+    params.set("input", inputDir);
+    params.set("output", outputDir);
+
+    String classificationMethod = "sequential";
+    classificationMethod = (String) cmdLine.getValue(methodOpt);
+    if (classificationMethod.equalsIgnoreCase("sequential"))
+      runFPGrowth(params);
+    else if (classificationMethod.equalsIgnoreCase("mapreduce"))
+      PFPGrowth.runPFPGrowth(params);
+  }
+
+  static void runFPGrowth(Parameters params) throws IOException {
+    log.info("Starting Sequential FPGrowth");
+    int maxHeapSize = Integer.valueOf(params.get("maxHeapSize", "50"));
+    int minSupport = Integer.valueOf(params.get("minSupport", "3"));
+
+    String output = params.get("output", "output.txt");
+
+    Path path = new Path(output);
+    Configuration conf = new Configuration();
+    FileSystem fs = FileSystem.get(conf);
+
+    Charset encoding = Charset.forName(params.get("encoding"));
+    String input = params.get("input");
+
+    String pattern = params.get("splitPattern", PFPGrowth.SPLITTER.toString());
+
+    SequenceFile.Writer writer = new SequenceFile.Writer(fs, conf, path,
+        Text.class, TopKStringPatterns.class);
+
+    FPGrowth<String> fp = new FPGrowth<String>();
+    Set<String> features = new HashSet<String>();
+
+    fp.generateTopKFrequentPatterns(new StringRecordIterator(
+        new FileLineIterable(new File(input), encoding, false), pattern), fp
+        .generateFList(new StringRecordIterator(new FileLineIterable(new File(
+            input), encoding, false), pattern), minSupport), minSupport,
+        maxHeapSize, features, new StringOutputConvertor(
+            new SequenceFileOutputCollector<Text, TopKStringPatterns>(writer)));
+    writer.close();
+
+    List<Pair<String, TopKStringPatterns>> frequentPatterns = FPGrowth
+        .readFrequentPattern(fs, conf, path);
+    for (Pair<String, TopKStringPatterns> entry : frequentPatterns)
+      log.info("Dumping Patterns for Feature: {} \n{}", entry.getFirst()
+          .toString(), entry.getSecond().toString());
+  }
+}