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 2010/02/13 20:57:24 UTC

svn commit: r909882 [5/5] - in /lucene/mahout/trunk/core/src/main/java/org/apache/mahout: classifier/ classifier/bayes/ classifier/bayes/algorithm/ classifier/bayes/common/ classifier/bayes/datastore/ classifier/bayes/exceptions/ classifier/bayes/inter...

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthMapper.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthMapper.java Sat Feb 13 19:57:04 2010
@@ -21,8 +21,8 @@
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -32,27 +32,22 @@
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
 
 /**
- * {@link ParallelFPGrowthMapper} maps each transaction to all unique items
- * groups in the transaction. mapper outputs the group id as key and the
- * transaction as value
+ * {@link ParallelFPGrowthMapper} maps each transaction to all unique items groups in the transaction. mapper
+ * outputs the group id as key and the transaction as value
  * 
  */
-public class ParallelFPGrowthMapper extends
-    Mapper<LongWritable,TransactionTree,LongWritable,TransactionTree> {
+public class ParallelFPGrowthMapper extends Mapper<LongWritable,TransactionTree,LongWritable,TransactionTree> {
   
   private final OpenIntLongHashMap gListInt = new OpenIntLongHashMap();
   
   @Override
-  protected void map(LongWritable offset,
-                     TransactionTree input,
-                     Context context) throws IOException,
-                                     InterruptedException {
+  protected void map(LongWritable offset, TransactionTree input, Context context) throws IOException,
+                                                                                 InterruptedException {
     
     Iterator<Pair<List<Integer>,Long>> it = input.getIterator();
     while (it.hasNext()) {
       Pair<List<Integer>,Long> pattern = it.next();
-      Integer[] prunedItems = pattern.getFirst().toArray(
-        new Integer[pattern.getFirst().size()]);
+      Integer[] prunedItems = pattern.getFirst().toArray(new Integer[pattern.getFirst().size()]);
       
       Set<Long> groups = new HashSet<Long>();
       for (int j = prunedItems.length - 1; j >= 0; j--) { // generate group
@@ -64,11 +59,8 @@
         if (groups.contains(groupID) == false) {
           Integer[] tempItems = new Integer[j + 1];
           System.arraycopy(prunedItems, 0, tempItems, 0, j + 1);
-          context.setStatus(
-            "Parallel FPGrowth: Generating Group Dependent transactions for: "
-            + item);
-          context.write(new LongWritable(groupID), new TransactionTree(
-              tempItems, pattern.getSecond()));
+          context.setStatus("Parallel FPGrowth: Generating Group Dependent transactions for: " + item);
+          context.write(new LongWritable(groupID), new TransactionTree(tempItems, pattern.getSecond()));
         }
         groups.add(groupID);
       }
@@ -77,21 +69,18 @@
   }
   
   @Override
-  protected void setup(Context context) throws IOException,
-                                       InterruptedException {
+  protected void setup(Context context) throws IOException, InterruptedException {
     super.setup(context);
-    Parameters params = Parameters.fromString(context.getConfiguration().get(
-      "pfp.parameters", ""));
+    Parameters params = Parameters.fromString(context.getConfiguration().get("pfp.parameters", ""));
     
     OpenObjectIntHashMap<String> fMap = new OpenObjectIntHashMap<String>();
     int i = 0;
-    for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList",
-      context.getConfiguration())) {
+    for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList", context.getConfiguration())) {
       fMap.put(e.getFirst(), i++);
     }
     
-    for (Entry<String,Long> e : PFPGrowth.deserializeMap(params, "gList",
-      context.getConfiguration()).entrySet()) {
+    for (Entry<String,Long> e : PFPGrowth.deserializeMap(params, "gList", context.getConfiguration())
+        .entrySet()) {
       gListInt.put(fMap.get(e.getKey()), e.getValue());
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/ParallelFPGrowthReducer.java Sat Feb 13 19:57:04 2010
@@ -44,14 +44,12 @@
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
 
 /**
- * {@link ParallelFPGrowthReducer} takes each group of transactions and runs
- * Vanilla FPGrowth on it and outputs the the Top K frequent Patterns for each
- * group.
+ * {@link ParallelFPGrowthReducer} takes each group of transactions and runs Vanilla FPGrowth on it and
+ * outputs the the Top K frequent Patterns for each group.
  * 
  */
 
-public class ParallelFPGrowthReducer extends
-    Reducer<LongWritable,TransactionTree,Text,TopKStringPatterns> {
+public class ParallelFPGrowthReducer extends Reducer<LongWritable,TransactionTree,Text,TopKStringPatterns> {
   
   private final List<Pair<Integer,Long>> fList = new ArrayList<Pair<Integer,Long>>();
   
@@ -68,9 +66,7 @@
   private int minSupport = 3;
   
   @Override
-  protected void reduce(LongWritable key,
-                        Iterable<TransactionTree> values,
-                        Context context) throws IOException {
+  protected void reduce(LongWritable key, Iterable<TransactionTree> values, Context context) throws IOException {
     TransactionTree cTree = new TransactionTree();
     int nodes = 0;
     for (TransactionTree tr : values) {
@@ -83,8 +79,7 @@
     
     List<Pair<Integer,Long>> localFList = new ArrayList<Pair<Integer,Long>>();
     for (Entry<Integer,MutableLong> fItem : cTree.generateFList().entrySet()) {
-      localFList.add(new Pair<Integer,Long>(fItem.getKey(), fItem.getValue()
-          .toLong()));
+      localFList.add(new Pair<Integer,Long>(fItem.getKey(), fItem.getValue().toLong()));
       
     }
     
@@ -102,31 +97,21 @@
     });
     
     FPGrowth<Integer> fpGrowth = new FPGrowth<Integer>();
-    fpGrowth
-        .generateTopKFrequentPatterns(
-          cTree.getIterator(),
-          localFList,
-          minSupport,
-          maxHeapSize,
-          new HashSet<Integer>(groupFeatures.get(key.get()).toList()),
-          new IntegerStringOutputConverter(
-              new ContextWriteOutputCollector<LongWritable,TransactionTree,Text,TopKStringPatterns>(
-                  context), featureReverseMap),
-          new ContextStatusUpdater<LongWritable,TransactionTree,Text,TopKStringPatterns>(
-              context));
+    fpGrowth.generateTopKFrequentPatterns(cTree.getIterator(), localFList, minSupport, maxHeapSize,
+      new HashSet<Integer>(groupFeatures.get(key.get()).toList()), new IntegerStringOutputConverter(
+          new ContextWriteOutputCollector<LongWritable,TransactionTree,Text,TopKStringPatterns>(context),
+          featureReverseMap), new ContextStatusUpdater<LongWritable,TransactionTree,Text,TopKStringPatterns>(
+          context));
   }
   
   @Override
-  protected void setup(Context context) throws IOException,
-                                       InterruptedException {
+  protected void setup(Context context) throws IOException, InterruptedException {
     
     super.setup(context);
-    Parameters params = Parameters.fromString(context.getConfiguration().get(
-      "pfp.parameters", ""));
+    Parameters params = Parameters.fromString(context.getConfiguration().get("pfp.parameters", ""));
     
     int i = 0;
-    for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList",
-      context.getConfiguration())) {
+    for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList", context.getConfiguration())) {
       featureReverseMap.add(e.getFirst());
       fMap.put(e.getFirst(), i);
       fRMap.add(e.getFirst());
@@ -134,8 +119,7 @@
       
     }
     
-    Map<String,Long> gList = PFPGrowth.deserializeMap(params, "gList", context
-        .getConfiguration());
+    Map<String,Long> gList = PFPGrowth.deserializeMap(params, "gList", context.getConfiguration());
     
     for (Entry<String,Long> entry : gList.entrySet()) {
       IntArrayList groupList = groupFeatures.get(entry.getValue());
@@ -151,8 +135,7 @@
     }
     maxHeapSize = Integer.valueOf(params.get("maxHeapSize", "50"));
     minSupport = Integer.valueOf(params.get("minSupport", "3"));
-    FPTreeDepthCache.setFirstLevelCacheSize(Integer.valueOf(params.get(
-      "treeCacheSize", Integer.toString(FPTreeDepthCache
-          .getFirstLevelCacheSize()))));
+    FPTreeDepthCache.setFirstLevelCacheSize(Integer.valueOf(params.get("treeCacheSize", Integer
+        .toString(FPTreeDepthCache.getFirstLevelCacheSize()))));
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingMapper.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingMapper.java Sat Feb 13 19:57:04 2010
@@ -34,13 +34,11 @@
 import org.apache.mahout.math.map.OpenObjectIntHashMap;
 
 /**
- * {@link TransactionSortingMapper} maps each transaction to all unique items
- * groups in the transaction. mapper outputs the group id as key and the
- * transaction as value
+ * {@link TransactionSortingMapper} maps each transaction to all unique items groups in the transaction.
+ * mapper outputs the group id as key and the transaction as value
  * 
  */
-public class TransactionSortingMapper extends
-    Mapper<LongWritable,Text,LongWritable,TransactionTree> {
+public class TransactionSortingMapper extends Mapper<LongWritable,Text,LongWritable,TransactionTree> {
   
   private final OpenObjectIntHashMap<String> fMap = new OpenObjectIntHashMap<String>();
   
@@ -65,27 +63,22 @@
     Integer[] prunedItems = itemSet.toArray(new Integer[itemSet.size()]);
     
     if (prunedItems.length > 0) {
-      context.write(new LongWritable(prunedItems[0]), new TransactionTree(
-          prunedItems, 1L));
+      context.write(new LongWritable(prunedItems[0]), new TransactionTree(prunedItems, 1L));
     }
     
   }
   
   @Override
-  protected void setup(Context context) throws IOException,
-                                       InterruptedException {
+  protected void setup(Context context) throws IOException, InterruptedException {
     super.setup(context);
-    Parameters params = Parameters.fromString(context.getConfiguration().get(
-      "pfp.parameters", ""));
+    Parameters params = Parameters.fromString(context.getConfiguration().get("pfp.parameters", ""));
     
     int i = 0;
-    for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList",
-      context.getConfiguration())) {
+    for (Pair<String,Long> e : PFPGrowth.deserializeList(params, "fList", context.getConfiguration())) {
       fMap.put(e.getFirst(), i++);
     }
     
-    splitter = Pattern.compile(params.get("splitPattern", PFPGrowth.SPLITTER
-        .toString()));
+    splitter = Pattern.compile(params.get("splitPattern", PFPGrowth.SPLITTER.toString()));
     
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingReducer.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionSortingReducer.java Sat Feb 13 19:57:04 2010
@@ -23,9 +23,8 @@
 import org.apache.hadoop.mapreduce.Reducer;
 
 /**
- * {@link TransactionSortingReducer} takes each group of transactions and runs
- * Vanilla FPGrowth on it and outputs the the Top K frequent Patterns for each
- * group.
+ * {@link TransactionSortingReducer} takes each group of transactions and runs Vanilla FPGrowth on it and
+ * outputs the the Top K frequent Patterns for each group.
  * 
  */
 
@@ -35,12 +34,10 @@
   private static final LongWritable ONE = new LongWritable(1);
   
   @Override
-  protected void reduce(LongWritable key,
-                        Iterable<TransactionTree> values,
-                        Context context) throws IOException,
-                                        InterruptedException {
+  protected void reduce(LongWritable key, Iterable<TransactionTree> values, Context context) throws IOException,
+                                                                                            InterruptedException {
     for (TransactionTree tr : values) {
-      context.write(ONE, tr);
+      context.write(TransactionSortingReducer.ONE, tr);
     }
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionTree.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionTree.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionTree.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/TransactionTree.java Sat Feb 13 19:57:04 2010
@@ -40,17 +40,16 @@
 
 /**
  * A compact representation of transactions modeled on the lines to
- * {@link org.apache.mahout.fpm.pfpgrowth.fpgrowth.FPTree} This reduces plenty of space and speeds up Map/Reduce of
- * {@link PFPGrowth} algorithm by reducing data size passed from the Mapper to
- * the reducer where {@link org.apache.mahout.fpm.pfpgrowth.fpgrowth.FPGrowth} mining is done
+ * {@link org.apache.mahout.fpm.pfpgrowth.fpgrowth.FPTree} This reduces plenty of space and speeds up
+ * Map/Reduce of {@link PFPGrowth} algorithm by reducing data size passed from the Mapper to the reducer where
+ * {@link org.apache.mahout.fpm.pfpgrowth.fpgrowth.FPGrowth} mining is done
  */
 public final class TransactionTree implements Writable {
   /**
-   * Generates a List of transactions view of Transaction Tree by doing Depth
-   * First Traversal on the tree structure
+   * Generates a List of transactions view of Transaction Tree by doing Depth First Traversal on the tree
+   * structure
    */
-  public final class TransactionTreeIterator implements
-      Iterator<Pair<List<Integer>,Long>> {
+  public final class TransactionTreeIterator implements Iterator<Pair<List<Integer>,Long>> {
     
     private final Stack<int[]> depth = new Stack<int[]>();
     
@@ -94,8 +93,7 @@
         data.add(attribute[it.next()[0]]);
       }
       
-      Pair<List<Integer>,Long> returnable = new Pair<List<Integer>,Long>(data,
-          nodeCount[childId] - sum);
+      Pair<List<Integer>,Long> returnable = new Pair<List<Integer>,Long>(data, nodeCount[childId] - sum);
       
       int[] top = depth.peek();
       while (top[1] + 1 == childCount[top[0]]) {
@@ -121,8 +119,7 @@
   
   private static final float GROWTH_RATE = 1.5f;
   
-  private static final Logger log = LoggerFactory
-      .getLogger(TransactionTree.class);
+  private static final Logger log = LoggerFactory.getLogger(TransactionTree.class);
   
   private static final int ROOTNODEID = 0;
   
@@ -141,13 +138,13 @@
   private List<Pair<List<Integer>,Long>> transactionSet = new ArrayList<Pair<List<Integer>,Long>>();
   
   public TransactionTree() {
-    this(DEFAULT_INITIAL_SIZE);
+    this(TransactionTree.DEFAULT_INITIAL_SIZE);
     representedAsList = false;
   }
   
   public TransactionTree(int size) {
-    if (size < DEFAULT_INITIAL_SIZE) {
-      size = DEFAULT_INITIAL_SIZE;
+    if (size < TransactionTree.DEFAULT_INITIAL_SIZE) {
+      size = TransactionTree.DEFAULT_INITIAL_SIZE;
     }
     childCount = new int[size];
     attribute = new int[size];
@@ -159,8 +156,7 @@
   
   public TransactionTree(Integer[] items, Long support) {
     representedAsList = true;
-    transactionSet.add(new Pair<List<Integer>,Long>(Arrays.asList(items),
-        support));
+    transactionSet.add(new Pair<List<Integer>,Long>(Arrays.asList(items), support));
   }
   
   public TransactionTree(List<Pair<List<Integer>,Long>> transactionSet) {
@@ -187,7 +183,7 @@
   }
   
   public int addPattern(List<Integer> myList, long addCount) {
-    int temp = ROOTNODEID;
+    int temp = TransactionTree.ROOTNODEID;
     int ret = 0;
     boolean addCountMode = true;
     for (int attributeValue : myList) {
@@ -290,14 +286,13 @@
       size += p.getFirst().size() + 2;
     }
     
-    log.debug("Nodes in UnCompressed Tree: {} ", nodes);
-    log.debug("UnCompressed Tree Size: {}", (this.nodes * 4 * 4 + this
-        .childCount() * 4)
-                                            / (double) 1000000);
-    log.debug("Nodes in Compressed Tree: {} ", node);
-    log.debug("Compressed Tree Size: {}",
-      (node * 4 * 4 + ctree.childCount() * 4) / (double) 1000000);
-    log.debug("TransactionSet Size: {}", (size * 4) / (double) 1000000);
+    TransactionTree.log.debug("Nodes in UnCompressed Tree: {} ", nodes);
+    TransactionTree.log.debug("UnCompressed Tree Size: {}", (this.nodes * 4 * 4 + this.childCount() * 4)
+                                                            / (double) 1000000);
+    TransactionTree.log.debug("Nodes in Compressed Tree: {} ", node);
+    TransactionTree.log.debug("Compressed Tree Size: {}", (node * 4 * 4 + ctree.childCount() * 4)
+                                                          / (double) 1000000);
+    TransactionTree.log.debug("TransactionSet Size: {}", size * 4 / (double) 1000000);
     if (node * 4 * 4 + ctree.childCount() * 4 <= size * 4) {
       return ctree;
     } else {
@@ -308,8 +303,7 @@
   
   public Iterator<Pair<List<Integer>,Long>> getIterator() {
     if (this.isTreeEmpty() && !representedAsList) {
-      throw new IllegalStateException(
-          "This is a bug. Please report this to mahout-user list");
+      throw new IllegalStateException("This is a bug. Please report this to mahout-user list");
     } else if (representedAsList) {
       return transactionSet.iterator();
     } else {
@@ -344,8 +338,7 @@
           vInt.readFields(in);
           items[j] = vInt.get();
         }
-        Pair<List<Integer>,Long> transaction = new Pair<List<Integer>,Long>(
-            Arrays.asList(items), support);
+        Pair<List<Integer>,Long> transaction = new Pair<List<Integer>,Long>(Arrays.asList(items), support);
         transactionSet.add(transaction);
       }
     } else {
@@ -420,7 +413,7 @@
     this.attribute[nodes] = attributeValue;
     nodeCount[nodes] = count;
     if (nodeChildren[nodes] == null) {
-      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+      nodeChildren[nodes] = new int[TransactionTree.DEFAULT_CHILDREN_INITIAL_SIZE];
     }
     
     int childNodeId = nodes++;
@@ -433,15 +426,15 @@
     attribute[nodes] = -1;
     nodeCount[nodes] = 0;
     if (nodeChildren[nodes] == null) {
-      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+      nodeChildren[nodes] = new int[TransactionTree.DEFAULT_CHILDREN_INITIAL_SIZE];
     }
     return nodes++;
   }
   
   private void resize() {
-    int size = (int) (GROWTH_RATE * nodes);
-    if (size < DEFAULT_INITIAL_SIZE) {
-      size = DEFAULT_INITIAL_SIZE;
+    int size = (int) (TransactionTree.GROWTH_RATE * nodes);
+    if (size < TransactionTree.DEFAULT_INITIAL_SIZE) {
+      size = TransactionTree.DEFAULT_INITIAL_SIZE;
     }
     
     int[] oldChildCount = childCount;
@@ -462,9 +455,9 @@
   
   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 size = (int) (TransactionTree.GROWTH_RATE * length);
+    if (size < TransactionTree.DEFAULT_CHILDREN_INITIAL_SIZE) {
+      size = TransactionTree.DEFAULT_CHILDREN_INITIAL_SIZE;
     }
     int[] oldNodeChildren = nodeChildren[nodeId];
     nodeChildren[nodeId] = new int[size];

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextStatusUpdater.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextStatusUpdater.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextStatusUpdater.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextStatusUpdater.java Sat Feb 13 19:57:04 2010
@@ -19,16 +19,17 @@
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.Reducer;
+
 /**
  * Updates the Context object of a {@link Reducer} class
- *
+ * 
  * @param <IK>
  * @param <IV>
  * @param <K>
  * @param <V>
  */
-public class ContextStatusUpdater<IK extends Writable,IV extends Writable,
-    K extends Writable,V extends Writable> implements StatusUpdater {
+public class ContextStatusUpdater<IK extends Writable,IV extends Writable,K extends Writable,V extends Writable>
+    implements StatusUpdater {
   
   private static final long PERIOD = 10000; // Update every 10 seconds
   
@@ -43,7 +44,7 @@
   @Override
   public void update(String status) {
     long curTime = System.currentTimeMillis();
-    if (curTime - time > PERIOD && context != null) {
+    if (curTime - time > ContextStatusUpdater.PERIOD && context != null) {
       time = curTime;
       context.setStatus("Processing FPTree: " + status);
     }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextWriteOutputCollector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextWriteOutputCollector.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextWriteOutputCollector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/ContextWriteOutputCollector.java Sat Feb 13 19:57:04 2010
@@ -26,8 +26,8 @@
 import org.slf4j.LoggerFactory;
 
 /**
- * An output collecter for {@link Reducer} for PFPGrowth which updates the
- * status as well as writes the patterns generated by the algorithm
+ * An output collecter for {@link Reducer} for PFPGrowth which updates the status as well as writes the
+ * patterns generated by the algorithm
  * 
  * @param <IK>
  * @param <IV>
@@ -37,8 +37,7 @@
 public class ContextWriteOutputCollector<IK extends Writable,IV extends Writable,K extends Writable,V extends Writable>
     implements OutputCollector<K,V> {
   
-  private static final Logger log = LoggerFactory
-      .getLogger(ContextWriteOutputCollector.class);
+  private static final Logger log = LoggerFactory.getLogger(ContextWriteOutputCollector.class);
   
   private final Reducer<IK,IV,K,V>.Context context;
   
@@ -52,7 +51,7 @@
       context.setStatus("Writing Top K patterns for: " + key.toString());
       context.write(key, value);
     } catch (InterruptedException e) {
-      log.error("{}", e);
+      ContextWriteOutputCollector.log.error("{}", e);
     }
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/SequenceFileOutputCollector.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/SequenceFileOutputCollector.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/SequenceFileOutputCollector.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/SequenceFileOutputCollector.java Sat Feb 13 19:57:04 2010
@@ -24,14 +24,13 @@
 import org.apache.hadoop.mapred.OutputCollector;
 
 /**
- * Collects the {@link Writable} key and {@link Writable} value, and writes them
- * into a {@link SequenceFile}
+ * Collects the {@link Writable} key and {@link Writable} value, and writes them into a {@link SequenceFile}
  * 
  * @param <K>
  * @param <V>
  */
-public class SequenceFileOutputCollector<K extends Writable,V extends Writable>
-    implements OutputCollector<K,V> {
+public class SequenceFileOutputCollector<K extends Writable,V extends Writable> implements
+    OutputCollector<K,V> {
   private final SequenceFile.Writer writer;
   
   public SequenceFileOutputCollector(SequenceFile.Writer writer) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TopKPatternsOutputConverter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TopKPatternsOutputConverter.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TopKPatternsOutputConverter.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TopKPatternsOutputConverter.java Sat Feb 13 19:57:04 2010
@@ -30,13 +30,13 @@
 import org.apache.mahout.fpm.pfpgrowth.fpgrowth.Pattern;
 
 /**
- * An output converter which converts the output patterns and collectes them in
- * a {@link FrequentPatternMaxHeap}
+ * An output converter which converts the output patterns and collectes them in a
+ * {@link FrequentPatternMaxHeap}
  * 
  * @param <A>
  */
-public final class TopKPatternsOutputConverter<A extends Comparable<? super A>>
-    implements OutputCollector<Integer,FrequentPatternMaxHeap> {
+public final class TopKPatternsOutputConverter<A extends Comparable<? super A>> implements
+    OutputCollector<Integer,FrequentPatternMaxHeap> {
   
   private OutputCollector<A,List<Pair<List<A>,Long>>> collector = null;
   
@@ -60,8 +60,7 @@
       }
       Collections.sort(frequentPattern);
       
-      Pair<List<A>,Long> returnItemSet = new Pair<List<A>,Long>(
-          frequentPattern, itemSet.support());
+      Pair<List<A>,Long> returnItemSet = new Pair<List<A>,Long>(frequentPattern, itemSet.support());
       perAttributePatterns.add(returnItemSet);
     }
     Collections.reverse(perAttributePatterns);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TransactionIterator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TransactionIterator.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TransactionIterator.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/TransactionIterator.java Sat Feb 13 19:57:04 2010
@@ -24,8 +24,8 @@
 import org.apache.mahout.common.Pair;
 
 /**
- * Iterates over a Transaction and outputs the transaction integer id mapping
- * and the support of the transaction
+ * Iterates over a Transaction and outputs the transaction integer id mapping and the support of the
+ * transaction
  * 
  * @param <AP>
  */
@@ -36,8 +36,7 @@
   
   private final int[] transactionBuffer;
   
-  public TransactionIterator(Iterator<Pair<List<AP>,Long>> iterator,
-                             Map<AP,Integer> attributeIdMapping) {
+  public TransactionIterator(Iterator<Pair<List<AP>,Long>> iterator, Map<AP,Integer> attributeIdMapping) {
     this.attributeIdMapping = attributeIdMapping;
     this.iterator = iterator;
     transactionBuffer = new int[attributeIdMapping.size()];

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/integer/IntegerStringOutputConverter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/integer/IntegerStringOutputConverter.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/integer/IntegerStringOutputConverter.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/integer/IntegerStringOutputConverter.java Sat Feb 13 19:57:04 2010
@@ -20,14 +20,15 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+
 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.string.TopKStringPatterns;
 
 /**
- * Collects the Patterns with Integer id and Long support and converts them to
- * Pattern of Strings based on a reverse feature lookup map.
+ * Collects the Patterns with Integer id and Long support and converts them to Pattern of Strings based on a
+ * reverse feature lookup map.
  */
 public final class IntegerStringOutputConverter implements
     OutputCollector<Integer,List<Pair<List<Integer>,Long>>> {
@@ -54,8 +55,7 @@
       stringValues.add(new Pair<List<String>,Long>(pattern, e.getSecond()));
     }
     
-    collector
-        .collect(new Text(stringKey), new TopKStringPatterns(stringValues));
+    collector.collect(new Text(stringKey), new TopKStringPatterns(stringValues));
   }
   
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringOutputConverter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringOutputConverter.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringOutputConverter.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringOutputConverter.java Sat Feb 13 19:57:04 2010
@@ -29,7 +29,7 @@
  * 
  */
 public final class StringOutputConverter implements
-    OutputCollector<String,List<Pair<List<String>,Long>>> {
+OutputCollector<String,List<Pair<List<String>,Long>>> {
   
   private final OutputCollector<Text,TopKStringPatterns> collector;
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringTupleIterator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringTupleIterator.java?rev=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringTupleIterator.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/convertors/string/StringTupleIterator.java Sat Feb 13 19:57:04 2010
@@ -27,27 +27,27 @@
  *
  */
 public final class StringTupleIterator implements Iterator<List<String>> {
-
+  
   private final Iterator<StringTuple> iterator;
-
+  
   public StringTupleIterator(Iterator<StringTuple> iterator) {
     this.iterator = iterator;
   }
-
+  
   @Override
   public boolean hasNext() {
     return iterator.hasNext();
   }
-
+  
   @Override
   public List<String> next() {
     StringTuple transaction = iterator.next();
     return transaction.getEntries();
   }
-
+  
   @Override
   public void remove() {
     iterator.remove();
   }
-
+  
 }

Modified: 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=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPGrowth.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPGrowth.java Sat Feb 13 19:57:04 2010
@@ -27,8 +27,8 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.commons.lang.mutable.MutableLong;
 import org.apache.hadoop.conf.Configuration;
@@ -57,8 +57,8 @@
   private static final Logger log = LoggerFactory.getLogger(FPGrowth.class);
   
   public static List<Pair<String,TopKStringPatterns>> readFrequentPattern(FileSystem fs,
-                                                                          Configuration conf,
-                                                                          Path path) throws IOException {
+    Configuration conf,
+    Path path) throws IOException {
     
     List<Pair<String,TopKStringPatterns>> ret = new ArrayList<Pair<String,TopKStringPatterns>>();
     Text key = new Text();
@@ -83,7 +83,7 @@
    * @return the List of features and their associated frequency as a Pair
    */
   public final List<Pair<A,Long>> generateFList(Iterator<Pair<List<A>,Long>> transactions,
-                                                int minSupport) {
+    int minSupport) {
     
     Map<A,MutableLong> attributeSupport = new HashMap<A,MutableLong>();
     // int count = 0;
@@ -92,7 +92,7 @@
       for (A attribute : transaction.getFirst()) {
         if (attributeSupport.containsKey(attribute) == false) {
           attributeSupport.put(attribute, new MutableLong(transaction
-              .getSecond()));
+            .getSecond()));
         } else {
           attributeSupport.get(attribute).add(
             transaction.getSecond().longValue());
@@ -174,15 +174,15 @@
       attributeFrequency[attributeIdMapping.get(attrib)] = frequency;
     }
     
-    log.info("Number of unique items {}", frequencyList.size());
+    FPGrowth.log.info("Number of unique items {}", frequencyList.size());
     
     Set<Integer> returnFeatures = new HashSet<Integer>();
     if (returnableFeatures.isEmpty() == false) {
       for (A attrib : returnableFeatures) {
         if (attributeIdMapping.containsKey(attrib)) {
           returnFeatures.add(attributeIdMapping.get(attrib));
-          log.info("Adding Pattern {}=>{}", attrib, attributeIdMapping
-              .get(attrib));
+          FPGrowth.log.info("Adding Pattern {}=>{}", attrib, attributeIdMapping
+            .get(attrib));
         }
       }
     } else {
@@ -191,11 +191,11 @@
       }
     }
     
-    log.info("Number of unique pruned items {}", attributeIdMapping.size());
+    FPGrowth.log.info("Number of unique pruned items {}", attributeIdMapping.size());
     generateTopKFrequentPatterns(new TransactionIterator<A>(transactionStream,
         attributeIdMapping), attributeFrequency, minSupport, k, reverseMapping
         .size(), returnFeatures, new TopKPatternsOutputConverter<A>(output,
-        reverseMapping), updater);
+            reverseMapping), updater);
     
   }
   
@@ -216,11 +216,11 @@
    * @return Top K Frequent Patterns for each feature and their support
    */
   private Map<Integer,FrequentPatternMaxHeap> fpGrowth(FPTree tree,
-                                                       MutableLong minSupportMutable,
-                                                       int k,
-                                                       Set<Integer> requiredFeatures,
-                                                       TopKPatternsOutputConverter<A> outputCollector,
-                                                       StatusUpdater updater) throws IOException {
+    MutableLong minSupportMutable,
+    int k,
+    Set<Integer> requiredFeatures,
+    TopKPatternsOutputConverter<A> outputCollector,
+    StatusUpdater updater) throws IOException {
     
     long minSupportValue = minSupportMutable.longValue();
     
@@ -231,18 +231,18 @@
       if (requiredFeatures.contains(attribute) == false) {
         continue;
       }
-      log.info("Mining FTree Tree for all patterns with {}", attribute);
+      FPGrowth.log.info("Mining FTree Tree for all patterns with {}", attribute);
       MutableLong minSupport = new MutableLong(minSupportValue);
-      FrequentPatternMaxHeap frequentPatterns = growth(tree, minSupport, k,
+      FrequentPatternMaxHeap frequentPatterns = FPGrowth.growth(tree, minSupport, k,
         treeCache, 0, attribute, updater);
       patterns.put(attribute, frequentPatterns);
       outputCollector.collect(attribute, frequentPatterns);
       
       minSupportValue = Math.max(minSupportValue, minSupport.longValue() / 2);
-      log.info("Found {} Patterns with Least Support {}", patterns.get(
+      FPGrowth.log.info("Found {} Patterns with Least Support {}", patterns.get(
         attribute).count(), patterns.get(attribute).leastSupport());
     }
-    log.info("Tree Cache: First Level: Cache hits={} Cache Misses={}",
+    FPGrowth.log.info("Tree Cache: First Level: Cache hits={} Cache Misses={}",
       treeCache.getHits(), treeCache.getMisses());
     return patterns;
   }
@@ -251,13 +251,13 @@
                                                                    int k,
                                                                    MutableLong minSupportMutable) {
     FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(k,
-        false);
+      false);
     
     int tempNode = FPTree.ROOTNODEID;
     Pattern frequentItem = new Pattern();
     while (tree.childCount(tempNode) != 0) {
       if (tree.childCount(tempNode) > 1) {
-        log.info("This should not happen {} {}", tree.childCount(tempNode),
+        FPGrowth.log.info("This should not happen {} {}", tree.childCount(tempNode),
           tempNode);
       }
       tempNode = tree.childAtIndex(tempNode, 0);
@@ -295,13 +295,13 @@
    * @return Top K frequent patterns for each attribute
    */
   private Map<Integer,FrequentPatternMaxHeap> generateTopKFrequentPatterns(Iterator<Pair<int[],Long>> transactions,
-                                                                           long[] attributeFrequency,
-                                                                           long minSupport,
-                                                                           int k,
-                                                                           int featureSetSize,
-                                                                           Set<Integer> returnFeatures,
-                                                                           TopKPatternsOutputConverter<A> topKPatternsOutputCollector,
-                                                                           StatusUpdater updater) throws IOException {
+    long[] attributeFrequency,
+    long minSupport,
+    int k,
+    int featureSetSize,
+    Set<Integer> returnFeatures,
+    TopKPatternsOutputConverter<A> topKPatternsOutputCollector,
+    StatusUpdater updater) throws IOException {
     
     FPTree tree = new FPTree(featureSetSize);
     for (int i = 0; i < featureSetSize; i++) {
@@ -317,15 +317,15 @@
       Pair<int[],Long> transaction = transactions.next();
       Arrays.sort(transaction.getFirst());
       // attribcount += transaction.length;
-      nodecount += treeAddCount(tree, transaction.getFirst(), transaction
-          .getSecond(), minSupportMutable, attributeFrequency);
+      nodecount += FPGrowth.treeAddCount(tree, transaction.getFirst(), transaction
+        .getSecond(), minSupportMutable, attributeFrequency);
       i++;
       if (i % 10000 == 0) {
-        log.info("FPTree Building: Read {} Transactions", i);
+        FPGrowth.log.info("FPTree Building: Read {} Transactions", i);
       }
     }
     
-    log.info("Number of Nodes in the FP Tree: {}", nodecount);
+    FPGrowth.log.info("Number of Nodes in the FP Tree: {}", nodecount);
     
     return fpGrowth(tree, minSupportMutable, k, returnFeatures,
       topKPatternsOutputCollector, updater);
@@ -340,7 +340,7 @@
                                                StatusUpdater updater) {
     
     FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(k,
-        true);
+      true);
     
     int i = Arrays.binarySearch(tree.getHeaderTableAttributes(),
       currentAttribute);
@@ -360,7 +360,7 @@
       updater.update("FPGrowth Algorithm for a given feature: " + attribute);
       FPTree conditionalTree = treeCache.getFirstLevelTree(attribute);
       if (conditionalTree.isEmpty()) {
-        traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+        FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
           minSupportMutable, conditionalTree, tree);
         // printTree(conditionalTree);
         
@@ -369,15 +369,15 @@
       FrequentPatternMaxHeap returnedPatterns;
       if (attribute == currentAttribute) {
         
-        returnedPatterns = growthTopDown(conditionalTree, minSupportMutable, k,
+        returnedPatterns = FPGrowth.growthTopDown(conditionalTree, minSupportMutable, k,
           treeCache, level + 1, true, currentAttribute, updater);
         
-        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+        frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
           attribute, count, true, true);
       } else {
-        returnedPatterns = growthTopDown(conditionalTree, minSupportMutable, k,
+        returnedPatterns = FPGrowth.growthTopDown(conditionalTree, minSupportMutable, k,
           treeCache, level + 1, false, currentAttribute, updater);
-        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+        frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
           attribute, count, false, true);
       }
       if (frequentPatterns.isFull()) {
@@ -401,7 +401,7 @@
                                                        StatusUpdater updater) {
     
     FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(k,
-        false);
+      false);
     
     if (conditionalOfCurrentAttribute == false) {
       int index = Arrays.binarySearch(tree.getHeaderTableAttributes(),
@@ -418,7 +418,7 @@
     }
     
     if (tree.singlePath()) {
-      return generateSinglePathPatterns(tree, k, minSupportMutable);
+      return FPGrowth.generateSinglePathPatterns(tree, k, minSupportMutable);
     }
     
     updater.update("Bottom Up FP Growth");
@@ -432,28 +432,28 @@
       
       FrequentPatternMaxHeap returnedPatterns;
       if (conditionalOfCurrentAttribute) {
-        traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+        FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
           minSupportMutable, conditionalTree, tree);
-        returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+        returnedPatterns = FPGrowth.growthBottomUp(conditionalTree, minSupportMutable,
           k, treeCache, level + 1, true, currentAttribute, updater);
         
-        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+        frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
           attribute, count, true, false);
       } else {
         if (attribute == currentAttribute) {
-          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+          FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
             minSupportMutable, conditionalTree, tree);
-          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+          returnedPatterns = FPGrowth.growthBottomUp(conditionalTree, minSupportMutable,
             k, treeCache, level + 1, true, currentAttribute, updater);
           
-          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+          frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
             attribute, count, true, false);
         } else if (attribute > currentAttribute) {
-          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+          FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
             minSupportMutable, conditionalTree, tree);
-          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+          returnedPatterns = FPGrowth.growthBottomUp(conditionalTree, minSupportMutable,
             k, treeCache, level + 1, false, currentAttribute, updater);
-          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+          frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
             attribute, count, false, false);
         }
       }
@@ -478,7 +478,7 @@
                                                       StatusUpdater updater) {
     
     FrequentPatternMaxHeap frequentPatterns = new FrequentPatternMaxHeap(k,
-        true);
+      true);
     
     if (conditionalOfCurrentAttribute == false) {
       int index = Arrays.binarySearch(tree.getHeaderTableAttributes(),
@@ -495,7 +495,7 @@
     }
     
     if (tree.singlePath()) {
-      return generateSinglePathPatterns(tree, k, minSupportMutable);
+      return FPGrowth.generateSinglePathPatterns(tree, k, minSupportMutable);
     }
     
     updater.update("Top Down Growth:");
@@ -511,29 +511,29 @@
       
       FrequentPatternMaxHeap returnedPatterns;
       if (conditionalOfCurrentAttribute) {
-        traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+        FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
           minSupportMutable, conditionalTree, tree);
         
-        returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+        returnedPatterns = FPGrowth.growthBottomUp(conditionalTree, minSupportMutable,
           k, treeCache, level + 1, true, currentAttribute, updater);
-        frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+        frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
           attribute, count, true, true);
         
       } else {
         if (attribute == currentAttribute) {
-          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+          FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
             minSupportMutable, conditionalTree, tree);
-          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+          returnedPatterns = FPGrowth.growthBottomUp(conditionalTree, minSupportMutable,
             k, treeCache, level + 1, true, currentAttribute, updater);
-          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+          frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
             attribute, count, true, false);
           
         } else if (attribute > currentAttribute) {
-          traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
+          FPGrowth.traverseAndBuildConditionalFPTreeData(tree.getHeaderNext(attribute),
             minSupportMutable, conditionalTree, tree);
-          returnedPatterns = growthBottomUp(conditionalTree, minSupportMutable,
+          returnedPatterns = FPGrowth.growthBottomUp(conditionalTree, minSupportMutable,
             k, treeCache, level + 1, false, currentAttribute, updater);
-          frequentPatterns = mergeHeap(frequentPatterns, returnedPatterns,
+          frequentPatterns = FPGrowth.mergeHeap(frequentPatterns, returnedPatterns,
             attribute, count, false, true);
           
         }
@@ -622,7 +622,7 @@
     
     tree.clearConditional();
     conditionalTree.reorderHeaderTable();
-    pruneFPTree(minSupportMutable, conditionalTree);
+    FPGrowth.pruneFPTree(minSupportMutable, conditionalTree);
     // prune Conditional Tree
     
   }

Modified: 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=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTree.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTree.java Sat Feb 13 19:57:04 2010
@@ -75,16 +75,16 @@
   private final Set<Integer> sortedSet = new TreeSet<Integer>();
   
   public FPTree() {
-    this(DEFAULT_INITIAL_SIZE, DEFAULT_HEADER_TABLE_INITIAL_SIZE);
+    this(FPTree.DEFAULT_INITIAL_SIZE, FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE);
   }
   
   public FPTree(int size) {
-    this(size, DEFAULT_HEADER_TABLE_INITIAL_SIZE);
+    this(size, FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE);
   }
   
   private FPTree(int size, int headersize) {
-    if (size < DEFAULT_INITIAL_SIZE) {
-      size = DEFAULT_INITIAL_SIZE;
+    if (size < FPTree.DEFAULT_INITIAL_SIZE) {
+      size = FPTree.DEFAULT_INITIAL_SIZE;
     }
     
     parent = new int[size];
@@ -96,11 +96,11 @@
     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];
+    headerTableAttributes = new int[FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE];
+    headerTableAttributeCount = new long[FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE];
+    headerTableLookup = new int[FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE];
     Arrays.fill(headerTableLookup, -1);
-    headerTableProperties = new int[DEFAULT_HEADER_TABLE_INITIAL_SIZE][];
+    headerTableProperties = new int[FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE][];
     
     singlePath = true;
     createRootNode();
@@ -134,12 +134,12 @@
   
   public final void addHeaderNext(int attributeValue, int nodeId) {
     int index = getHeaderIndex(attributeValue);
-    if (headerTableProperties[index][HT_NEXT] == -1) {
-      headerTableProperties[index][HT_NEXT] = nodeId;
-      headerTableProperties[index][HT_LAST] = nodeId;
+    if (headerTableProperties[index][FPTree.HT_NEXT] == -1) {
+      headerTableProperties[index][FPTree.HT_NEXT] = nodeId;
+      headerTableProperties[index][FPTree.HT_LAST] = nodeId;
     } else {
-      setNext(headerTableProperties[index][HT_LAST], nodeId);
-      headerTableProperties[index][HT_LAST] = nodeId;
+      setNext(headerTableProperties[index][FPTree.HT_LAST], nodeId);
+      headerTableProperties[index][FPTree.HT_LAST] = nodeId;
     }
   }
   
@@ -203,7 +203,7 @@
     nodeCount[nodes] = count;
     
     if (nodeChildren[nodes] == null) {
-      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+      nodeChildren[nodes] = new int[FPTree.DEFAULT_CHILDREN_INITIAL_SIZE];
     }
     
     return nodes++;
@@ -222,7 +222,7 @@
     
     conditional[nodes] = 0;
     if (nodeChildren[nodes] == null) {
-      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+      nodeChildren[nodes] = new int[FPTree.DEFAULT_CHILDREN_INITIAL_SIZE];
     }
     
     int childNodeId = nodes++;
@@ -238,7 +238,7 @@
     attribute[nodes] = -1;
     nodeCount[nodes] = 0;
     if (nodeChildren[nodes] == null) {
-      nodeChildren[nodes] = new int[DEFAULT_CHILDREN_INITIAL_SIZE];
+      nodeChildren[nodes] = new int[FPTree.DEFAULT_CHILDREN_INITIAL_SIZE];
     }
     return nodes++;
   }
@@ -249,7 +249,7 @@
   
   public final int getHeaderNext(int attributeValue) {
     int index = getHeaderIndex(attributeValue);
-    return headerTableProperties[index][HT_NEXT];
+    return headerTableProperties[index][FPTree.HT_NEXT];
   }
   
   public final long getHeaderSupportCount(int attributeValue) {
@@ -281,7 +281,7 @@
   
   public final void removeHeaderNext(int attributeValue) {
     int index = getHeaderIndex(attributeValue);
-    headerTableProperties[index][HT_NEXT] = -1;
+    headerTableProperties[index][FPTree.HT_NEXT] = -1;
   }
   
   public final void reorderHeaderTable() {
@@ -351,11 +351,11 @@
       }
       headerTableAttributes[headerTableCount] = attributeValue;
       if (headerTableProperties[headerTableCount] == null) {
-        headerTableProperties[headerTableCount] = new int[HEADERTABLEBLOCKSIZE];
+        headerTableProperties[headerTableCount] = new int[FPTree.HEADERTABLEBLOCKSIZE];
       }
       headerTableAttributeCount[headerTableCount] = 0;
-      headerTableProperties[headerTableCount][HT_NEXT] = -1;
-      headerTableProperties[headerTableCount][HT_LAST] = -1;
+      headerTableProperties[headerTableCount][FPTree.HT_NEXT] = -1;
+      headerTableProperties[headerTableCount][FPTree.HT_LAST] = -1;
       index = headerTableCount++;
       headerTableLookup[attributeValue] = index;
       sortedSet.add(attributeValue);
@@ -364,9 +364,9 @@
   }
   
   private void resize() {
-    int size = (int) (GROWTH_RATE * nodes);
-    if (size < DEFAULT_INITIAL_SIZE) {
-      size = DEFAULT_INITIAL_SIZE;
+    int size = (int) (FPTree.GROWTH_RATE * nodes);
+    if (size < FPTree.DEFAULT_INITIAL_SIZE) {
+      size = FPTree.DEFAULT_INITIAL_SIZE;
     }
     
     int[] oldChildCount = childCount;
@@ -397,9 +397,9 @@
   
   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 size = (int) (FPTree.GROWTH_RATE * length);
+    if (size < FPTree.DEFAULT_CHILDREN_INITIAL_SIZE) {
+      size = FPTree.DEFAULT_CHILDREN_INITIAL_SIZE;
     }
     int[] oldNodeChildren = nodeChildren[nodeId];
     nodeChildren[nodeId] = new int[size];
@@ -407,7 +407,7 @@
   }
   
   private void resizeHeaderLookup(int attributeValue) {
-    int size = (int) (attributeValue * GROWTH_RATE);
+    int size = (int) (attributeValue * FPTree.GROWTH_RATE);
     int[] oldLookup = headerTableLookup;
     headerTableLookup = new int[size];
     Arrays.fill(headerTableLookup, oldLookup.length, size, -1);
@@ -415,9 +415,9 @@
   }
   
   private void resizeHeaderTable() {
-    int size = (int) (GROWTH_RATE * headerTableCount);
-    if (size < DEFAULT_HEADER_TABLE_INITIAL_SIZE) {
-      size = DEFAULT_HEADER_TABLE_INITIAL_SIZE;
+    int size = (int) (FPTree.GROWTH_RATE * headerTableCount);
+    if (size < FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE) {
+      size = FPTree.DEFAULT_HEADER_TABLE_INITIAL_SIZE;
     }
     
     int[] oldAttributes = headerTableAttributes;

Modified: 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=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTreeDepthCache.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/FPTreeDepthCache.java Sat Feb 13 19:57:04 2010
@@ -34,10 +34,10 @@
   private static int firstLevelCacheSize = 5;
   
   private static final Logger log = LoggerFactory.getLogger(
-      FPTreeDepthCache.class);
+    FPTreeDepthCache.class);
   
-  private final LeastKCache<Integer,FPTree> firstLevelCache 
-      = new LeastKCache<Integer,FPTree>(firstLevelCacheSize);
+  private final LeastKCache<Integer,FPTree> firstLevelCache
+  = new LeastKCache<Integer,FPTree>(FPTreeDepthCache.firstLevelCacheSize);
   
   private int hits;
   
@@ -46,12 +46,12 @@
   private final List<FPTree> treeCache = new ArrayList<FPTree>();
   
   public FPTreeDepthCache() {
-    log.info("Initializing FPTreeCache with firstLevelCacheSize: {}",
-      firstLevelCacheSize);
+    FPTreeDepthCache.log.info("Initializing FPTreeCache with firstLevelCacheSize: {}",
+      FPTreeDepthCache.firstLevelCacheSize);
   }
   
   public static int getFirstLevelCacheSize() {
-    return firstLevelCacheSize;
+    return FPTreeDepthCache.firstLevelCacheSize;
   }
   
   public static void setFirstLevelCacheSize(int firstLevelCacheSize) {

Modified: 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=909882&r1=909881&r2=909882&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/Pattern.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/fpgrowth/Pattern.java Sat Feb 13 19:57:04 2010
@@ -43,12 +43,12 @@
   private long[] supportValues;
   
   public Pattern() {
-    this(DEFAULT_INITIAL_SIZE);
+    this(Pattern.DEFAULT_INITIAL_SIZE);
   }
   
   private Pattern(int size) {
-    if (size < DEFAULT_INITIAL_SIZE) {
-      size = DEFAULT_INITIAL_SIZE;
+    if (size < Pattern.DEFAULT_INITIAL_SIZE) {
+      size = Pattern.DEFAULT_INITIAL_SIZE;
     }
     this.pattern = new int[size];
     this.supportValues = new long[size];
@@ -61,7 +61,7 @@
     }
     this.pattern[length] = id;
     this.supportValues[length++] = supportCount;
-    this.support = (supportCount > this.support) ? this.support : supportCount;
+    this.support = supportCount > this.support ? this.support : supportCount;
     dirty = true;
   }
   
@@ -144,9 +144,9 @@
   }
   
   private void resize() {
-    int size = (int) (GROWTH_RATE * length);
-    if (size < DEFAULT_INITIAL_SIZE) {
-      size = DEFAULT_INITIAL_SIZE;
+    int size = (int) (Pattern.GROWTH_RATE * length);
+    if (size < Pattern.DEFAULT_INITIAL_SIZE) {
+      size = Pattern.DEFAULT_INITIAL_SIZE;
     }
     int[] oldpattern = pattern;
     long[] oldSupport = supportValues;