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/15 19:18:05 UTC

svn commit: r910282 [3/6] - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/cf/taste/hadoop/ core/src/main/java/org/apache/mahout/cf/taste/hadoop/cooccurence/ core/src/main/java/org/apache/mahout/cf/taste/hadoop/item/ core/src/main/java/o...

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonClusterAdapter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonClusterAdapter.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonClusterAdapter.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonClusterAdapter.java Mon Feb 15 18:17:54 2010
@@ -53,7 +53,9 @@
   }
   
   @Override
-  public DirichletCluster<?> deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context) throws JsonParseException {
+  public DirichletCluster<?> deserialize(JsonElement json,
+                                         Type typeOfT,
+                                         JsonDeserializationContext context) throws JsonParseException {
     GsonBuilder builder = new GsonBuilder();
     builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
     Gson gson = builder.create();
@@ -66,7 +68,7 @@
     try {
       cl = ccl.loadClass(klass);
     } catch (ClassNotFoundException e) {
-      JsonClusterAdapter.log.warn("Error while loading class", e);
+      log.warn("Error while loading class", e);
     }
     Model<Vector> model = (Model<Vector>) gson.fromJson(modelJson, cl);
     return new DirichletCluster<Vector>(model, total);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java Mon Feb 15 18:17:54 2010
@@ -51,7 +51,8 @@
   }
   
   @Override
-  public Model<?> deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context) throws JsonParseException {
+  public Model<?> deserialize(JsonElement json, Type typeOfT,
+                              JsonDeserializationContext context) throws JsonParseException {
     GsonBuilder builder = new GsonBuilder();
     builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
     Gson gson = builder.create();
@@ -63,7 +64,7 @@
     try {
       cl = ccl.loadClass(klass);
     } catch (ClassNotFoundException e) {
-      JsonModelAdapter.log.warn("Error while loading class", e);
+      log.warn("Error while loading class", e);
     }
     return (Model<?>) gson.fromJson(model, cl);
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansDriver.java Mon Feb 15 18:17:54 2010
@@ -184,7 +184,7 @@
       }
       
     } catch (OptionException e) {
-      FuzzyKMeansDriver.log.error("Exception", e);
+      log.error("Exception", e);
       CommandLineUtil.printHelp(group);
     }
     
@@ -228,7 +228,7 @@
     
     // iterate until the clusters converge
     while (!converged && (iteration < maxIterations)) {
-      FuzzyKMeansDriver.log.info("Iteration {}", iteration);
+      log.info("Iteration {}", iteration);
       
       // point the output to a new directory per iteration
       String clustersOut = output + File.separator + "clusters-" + iteration;
@@ -241,7 +241,7 @@
     }
     
     // now actually cluster the points
-    FuzzyKMeansDriver.log.info("Clustering ");
+    log.info("Clustering ");
     
     FuzzyKMeansDriver.runClustering(input, clustersIn, output + File.separator + "points", measureClass,
       convergenceDelta, numMapTasks, m);
@@ -313,7 +313,7 @@
       FileSystem fs = FileSystem.get(outPath.toUri(), conf);
       return FuzzyKMeansDriver.isConverged(clustersOut, conf, fs);
     } catch (IOException e) {
-      FuzzyKMeansDriver.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
       return true;
     }
   }
@@ -370,7 +370,7 @@
     try {
       JobClient.runJob(conf);
     } catch (IOException e) {
-      FuzzyKMeansDriver.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
     }
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansJob.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansJob.java Mon Feb 15 18:17:54 2010
@@ -105,7 +105,7 @@
       FuzzyKMeansJob.runJob(input, clusters, output, measureClass, convergenceDelta, maxIterations,
         numMapTasks, numReduceTasks, doCanopy, m);
     } catch (OptionException e) {
-      FuzzyKMeansJob.log.error("Exception parsing command line: ", e);
+      log.error("Exception parsing command line: ", e);
       CommandLineUtil.printHelp(group);
     }
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansMapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansMapper.java Mon Feb 15 18:17:54 2010
@@ -65,7 +65,7 @@
     super.configure(job);
     clusterer = new FuzzyKMeansClusterer(job);
     
-    FuzzyKMeansMapper.log.info("In Mapper Configure:");
+    log.info("In Mapper Configure:");
     
     String clusterPath = job.get(FuzzyKMeansConfigKeys.CLUSTER_PATH_KEY);
     if ((clusterPath != null) && (clusterPath.length() > 0)) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansUtil.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansUtil.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansUtil.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/fuzzykmeans/FuzzyKMeansUtil.java Mon Feb 15 18:17:54 2010
@@ -44,7 +44,7 @@
     Configuration job = new Configuration();
     Path clusterPath = new Path(clusterPathStr + "/*");
     List<Path> result = new ArrayList<Path>();
-    // log.info("I am here");
+    //log.info("I am here");
     // filter out the files
     PathFilter clusterFileFilter = new PathFilter() {
       @Override
@@ -75,10 +75,10 @@
           try {
             key = (Writable) reader.getKeyClass().newInstance();
           } catch (InstantiationException e) { // Should not be possible
-            FuzzyKMeansUtil.log.error("Exception", e);
+            log.error("Exception", e);
             throw new IllegalStateException(e);
           } catch (IllegalAccessException e) {
-            FuzzyKMeansUtil.log.error("Exception", e);
+            log.error("Exception", e);
             throw new IllegalStateException(e);
           }
           if (valueClass.equals(Cluster.class)) {
@@ -103,9 +103,9 @@
       }
       
     } catch (IOException e) {
-      FuzzyKMeansUtil.log.info("Exception occurred in loading clusters:", e);
+      log.info("Exception occurred in loading clusters:", e);
       throw new IllegalStateException(e);
     }
   }
   
-}
\ No newline at end of file
+}

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/Cluster.java Mon Feb 15 18:17:54 2010
@@ -54,7 +54,7 @@
   
   @Override
   public String asFormatString() {
-    return Cluster.formatCluster(this);
+    return formatCluster(this);
   }
   
   /**
@@ -69,7 +69,7 @@
   public static Cluster decodeCluster(String formattedString) {
     int beginIndex = formattedString.indexOf('{');
     if (beginIndex <= 0) {
-      throw new IllegalArgumentException(Cluster.ERROR_UNKNOWN_CLUSTER_FORMAT + formattedString);
+      throw new IllegalArgumentException(ERROR_UNKNOWN_CLUSTER_FORMAT + formattedString);
     }
     String id = formattedString.substring(0, beginIndex);
     String center = formattedString.substring(beginIndex);
@@ -82,7 +82,7 @@
       cluster = new Cluster(clusterCenter, clusterId);
       cluster.setConverged(startsWithV);
     } else {
-      throw new IllegalArgumentException(Cluster.ERROR_UNKNOWN_CLUSTER_FORMAT + formattedString);
+      throw new IllegalArgumentException(ERROR_UNKNOWN_CLUSTER_FORMAT + formattedString);
     }
     return cluster;
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansClusterer.java Mon Feb 15 18:17:54 2010
@@ -66,7 +66,7 @@
       Vector clusterCenter = cluster.getCenter();
       double distance = this.measure.distance(clusterCenter.getLengthSquared(), clusterCenter, point);
       if (KMeansClusterer.log.isDebugEnabled()) {
-        KMeansClusterer.log.debug("{} Cluster: {}", distance, cluster.getId());
+        log.debug("{} Cluster: {}", distance, cluster.getId());
       }
       if ((distance < nearestDistance) || (nearestCluster == null)) {
         nearestCluster = cluster;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansDriver.java Mon Feb 15 18:17:54 2010
@@ -52,7 +52,7 @@
   
   private static final Logger log = LoggerFactory.getLogger(KMeansDriver.class);
   
-  private KMeansDriver() { }
+  private KMeansDriver() {}
   
   /**
    * @param args
@@ -73,8 +73,10 @@
         .withRequired(true)
         .withArgument(abuilder.withName("clusters").withMinimum(1).withMaximum(1).create())
         .withDescription(
-          "The input centroids, as Vectors.  Must be a SequenceFile of Writable, Cluster/Canopy.  "
-              + "If k is also specified, then a random set of vectors will be selected and written out to this path first")
+          "The input centroids, as Vectors. "
+              + "Must be a SequenceFile of Writable, Cluster/Canopy.  "
+              + "If k is also specified, then a random set of vectors will be selected "
+              + "and written out to this path first")
         .withShortName("c").create();
     
     Option kOpt = obuilder
@@ -82,10 +84,11 @@
         .withRequired(false)
         .withArgument(abuilder.withName("k").withMinimum(1).withMaximum(1).create())
         .withDescription(
-          "The k in k-Means.  If specified, then a random selection of k Vectors will be chosen as the Centroid and written to the clusters output path.")
+          "The k in k-Means.  If specified, then a random selection of k Vectors will be chosen "
+              + "as the Centroid and written to the clusters output path.")
         .withShortName("k").create();
-    
-    Option outputOpt = obuilder.withLongName("output").withRequired(true).withArgument(
+
+   Option outputOpt = obuilder.withLongName("output").withRequired(true).withArgument(
       abuilder.withName("output").withMinimum(1).withMaximum(1).create()).withDescription(
       "The Path to put the output in").withShortName("o").create();
     
@@ -161,10 +164,9 @@
         clusters = RandomSeedGenerator.buildRandom(input, clusters,
           Integer.parseInt(cmdLine.getValue(kOpt).toString())).toString();
       }
-      KMeansDriver.runJob(input, clusters, output, measureClass, convergenceDelta, maxIterations,
-        numReduceTasks);
+      runJob(input, clusters, output, measureClass, convergenceDelta, maxIterations, numReduceTasks);
     } catch (OptionException e) {
-      KMeansDriver.log.error("Exception", e);
+      log.error("Exception", e);
       CommandLineUtil.printHelp(group);
     }
   }
@@ -196,30 +198,26 @@
                             int numReduceTasks) {
     // iterate until the clusters converge
     String delta = Double.toString(convergenceDelta);
-    if (KMeansDriver.log.isInfoEnabled()) {
-      KMeansDriver.log.info("Input: {} Clusters In: {} Out: {} Distance: {}", new Object[] {input,
-                                                                                            clustersIn,
-                                                                                            output,
-                                                                                            measureClass});
-      KMeansDriver.log.info("convergence: {} max Iterations: {} num Reduce Tasks: {} Input Vectors: {}",
+    if (log.isInfoEnabled()) {
+      log.info("Input: {} Clusters In: {} Out: {} Distance: {}", new Object[] {input, clustersIn, output,
+                                                                               measureClass});
+      log.info("convergence: {} max Iterations: {} num Reduce Tasks: {} Input Vectors: {}",
         new Object[] {convergenceDelta, maxIterations, numReduceTasks, VectorWritable.class.getName()});
     }
     boolean converged = false;
     int iteration = 0;
     while (!converged && (iteration < maxIterations)) {
-      KMeansDriver.log.info("Iteration {}", iteration);
+      log.info("Iteration {}", iteration);
       // point the output to a new directory per iteration
       String clustersOut = output + "/clusters-" + iteration;
-      converged = KMeansDriver.runIteration(input, clustersIn, clustersOut, measureClass, delta,
-        numReduceTasks, iteration);
+      converged = runIteration(input, clustersIn, clustersOut, measureClass, delta, numReduceTasks, iteration);
       // now point the input to the old output directory
       clustersIn = output + "/clusters-" + iteration;
       iteration++;
     }
     // now actually cluster the points
-    KMeansDriver.log.info("Clustering ");
-    KMeansDriver.runClustering(input, clustersIn, output + KMeansDriver.DEFAULT_OUTPUT_DIRECTORY,
-      measureClass, delta);
+    log.info("Clustering ");
+    runClustering(input, clustersIn, output + DEFAULT_OUTPUT_DIRECTORY, measureClass, delta);
   }
   
   /**
@@ -271,9 +269,9 @@
     try {
       JobClient.runJob(conf);
       FileSystem fs = FileSystem.get(outPath.toUri(), conf);
-      return KMeansDriver.isConverged(clustersOut, conf, fs);
+      return isConverged(clustersOut, conf, fs);
     } catch (IOException e) {
-      KMeansDriver.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
       return true;
     }
   }
@@ -297,14 +295,11 @@
                                     String output,
                                     String measureClass,
                                     String convergenceDelta) {
-    if (KMeansDriver.log.isInfoEnabled()) {
-      KMeansDriver.log.info("Running Clustering");
-      KMeansDriver.log.info("Input: {} Clusters In: {} Out: {} Distance: {}", new Object[] {input,
-                                                                                            clustersIn,
-                                                                                            output,
-                                                                                            measureClass});
-      KMeansDriver.log.info("convergence: {} Input Vectors: {}", convergenceDelta, VectorWritable.class
-          .getName());
+    if (log.isInfoEnabled()) {
+      log.info("Running Clustering");
+      log.info("Input: {} Clusters In: {} Out: {} Distance: {}", new Object[] {input, clustersIn, output,
+                                                                               measureClass});
+      log.info("convergence: {} Input Vectors: {}", convergenceDelta, VectorWritable.class.getName());
     }
     JobConf conf = new JobConf(KMeansDriver.class);
     conf.setInputFormat(SequenceFileInputFormat.class);
@@ -329,7 +324,7 @@
     try {
       JobClient.runJob(conf);
     } catch (IOException e) {
-      KMeansDriver.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
     }
   }
   
@@ -356,10 +351,10 @@
         try {
           key = (Writable) reader.getKeyClass().newInstance();
         } catch (InstantiationException e) { // shouldn't happen
-          KMeansDriver.log.error("Exception", e);
+          log.error("Exception", e);
           throw new IllegalStateException(e);
         } catch (IllegalAccessException e) {
-          KMeansDriver.log.error("Exception", e);
+          log.error("Exception", e);
           throw new IllegalStateException(e);
         }
         Cluster value = new Cluster();

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansUtil.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansUtil.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansUtil.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/KMeansUtil.java Mon Feb 15 18:17:54 2010
@@ -76,10 +76,10 @@
           try {
             key = (Writable) reader.getKeyClass().newInstance();
           } catch (InstantiationException e) { // Should not be possible
-            KMeansUtil.log.error("Exception", e);
+            log.error("Exception", e);
             throw new IllegalStateException(e);
           } catch (IllegalAccessException e) {
-            KMeansUtil.log.error("Exception", e);
+            log.error("Exception", e);
             throw new IllegalStateException(e);
           }
           if (valueClass.equals(Cluster.class)) {
@@ -104,7 +104,7 @@
       }
       
     } catch (IOException e) {
-      KMeansUtil.log.info("Exception occurred in loading clusters:", e);
+      log.info("Exception occurred in loading clusters:", e);
       throw new IllegalStateException(e);
     }
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/kmeans/RandomSeedGenerator.java Mon Feb 15 18:17:54 2010
@@ -61,7 +61,7 @@
     fs.mkdirs(outPath);
     Path outFile = new Path(outPath, "part-randomSeed");
     if (fs.exists(outFile)) {
-      RandomSeedGenerator.log.warn("Deleting {}", outFile);
+      log.warn("Deleting {}", outFile);
       fs.delete(outFile, false);
     }
     boolean newFile = fs.createNewFile(outFile);
@@ -111,7 +111,7 @@
       for (int i = 0; i < k; i++) {
         writer.append(chosenTexts.get(i), chosenClusters.get(i));
       }
-      RandomSeedGenerator.log.info("Wrote {} vectors to {}", k, outFile);
+      log.info("Wrote {} vectors to {}", k, outFile);
       writer.close();
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDADriver.java Mon Feb 15 18:17:54 2010
@@ -155,10 +155,10 @@
         topicSmoothing = 50.0 / numTopics;
       }
       
-      LDADriver.runJob(input, output, numTopics, numWords, topicSmoothing, maxIterations, numReduceTasks);
+      runJob(input, output, numTopics, numWords, topicSmoothing, maxIterations, numReduceTasks);
       
     } catch (OptionException e) {
-      LDADriver.log.error("Exception", e);
+      log.error("Exception", e);
       CommandLineUtil.printHelp(group);
     }
   }
@@ -191,24 +191,24 @@
                             int numReducers) throws IOException, InterruptedException, ClassNotFoundException {
     
     String stateIn = output + "/state-0";
-    LDADriver.writeInitialState(stateIn, numTopics, numWords);
+    writeInitialState(stateIn, numTopics, numWords);
     double oldLL = Double.NEGATIVE_INFINITY;
     boolean converged = false;
     
     for (int iteration = 0; ((maxIterations < 1) || (iteration < maxIterations)) && !converged; iteration++) {
-      LDADriver.log.info("Iteration {}", iteration);
+      log.info("Iteration {}", iteration);
       // point the output to a new directory per iteration
       String stateOut = output + "/state-" + (iteration + 1);
-      double ll = LDADriver.runIteration(input, stateIn, stateOut, numTopics, numWords, topicSmoothing,
+      double ll = runIteration(input, stateIn, stateOut, numTopics, numWords, topicSmoothing,
         numReducers);
       double relChange = (oldLL - ll) / oldLL;
       
       // now point the input to the old output directory
-      LDADriver.log.info("Iteration {} finished. Log Likelihood: {}", iteration, ll);
-      LDADriver.log.info("(Old LL: {})", oldLL);
-      LDADriver.log.info("(Rel Change: {})", relChange);
+      log.info("Iteration {} finished. Log Likelihood: {}", iteration, ll);
+      log.info("(Old LL: {})", oldLL);
+      log.info("(Rel Change: {})", relChange);
       
-      converged = (iteration > 2) && (relChange < LDADriver.OVERALL_CONVERGENCE);
+      converged = (iteration > 2) && (relChange < OVERALL_CONVERGENCE);
       stateIn = stateOut;
       oldLL = ll;
     }
@@ -240,7 +240,7 @@
         writer.append(kw, v);
       }
       
-      kw.setY(LDADriver.TOPIC_SUM_KEY);
+      kw.setY(TOPIC_SUM_KEY);
       v.set(Math.log(total));
       writer.append(kw, v);
       
@@ -260,7 +260,7 @@
       Path path = status.getPath();
       SequenceFile.Reader reader = new SequenceFile.Reader(fs, path, job);
       while (reader.next(key, value)) {
-        if (key.getX() == LDADriver.LOG_LIKELIHOOD_KEY) {
+        if (key.getX() == LOG_LIKELIHOOD_KEY) {
           ll = value.get();
           break;
         }
@@ -295,10 +295,10 @@
                                                     InterruptedException,
                                                     ClassNotFoundException {
     Configuration conf = new Configuration();
-    conf.set(LDADriver.STATE_IN_KEY, stateIn);
-    conf.set(LDADriver.NUM_TOPICS_KEY, Integer.toString(numTopics));
-    conf.set(LDADriver.NUM_WORDS_KEY, Integer.toString(numWords));
-    conf.set(LDADriver.TOPIC_SMOOTHING_KEY, Double.toString(topicSmoothing));
+    conf.set(STATE_IN_KEY, stateIn);
+    conf.set(NUM_TOPICS_KEY, Integer.toString(numTopics));
+    conf.set(NUM_WORDS_KEY, Integer.toString(numWords));
+    conf.set(TOPIC_SMOOTHING_KEY, Double.toString(topicSmoothing));
     
     Job job = new Job(conf);
     
@@ -317,14 +317,14 @@
     job.setJarByClass(LDADriver.class);
     
     job.waitForCompletion(true);
-    return LDADriver.findLL(stateOut, conf);
+    return findLL(stateOut, conf);
   }
   
   static LDAState createState(Configuration job) throws IOException {
-    String statePath = job.get(LDADriver.STATE_IN_KEY);
-    int numTopics = Integer.parseInt(job.get(LDADriver.NUM_TOPICS_KEY));
-    int numWords = Integer.parseInt(job.get(LDADriver.NUM_WORDS_KEY));
-    double topicSmoothing = Double.parseDouble(job.get(LDADriver.TOPIC_SMOOTHING_KEY));
+    String statePath = job.get(STATE_IN_KEY);
+    int numTopics = Integer.parseInt(job.get(NUM_TOPICS_KEY));
+    int numWords = Integer.parseInt(job.get(NUM_WORDS_KEY));
+    double topicSmoothing = Double.parseDouble(job.get(TOPIC_SMOOTHING_KEY));
     
     Path dir = new Path(statePath);
     FileSystem fs = dir.getFileSystem(job);
@@ -341,12 +341,12 @@
       while (reader.next(key, value)) {
         int topic = key.getX();
         int word = key.getY();
-        if (word == LDADriver.TOPIC_SUM_KEY) {
+        if (word == TOPIC_SUM_KEY) {
           logTotals[topic] = value.get();
           if (Double.isInfinite(value.get())) {
             throw new IllegalArgumentException();
           }
-        } else if (topic == LDADriver.LOG_LIKELIHOOD_KEY) {
+        } else if (topic == LOG_LIKELIHOOD_KEY) {
           ll = value.get();
         } else {
           if (!((topic >= 0) && (word >= 0))) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAInference.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAInference.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAInference.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAInference.java Mon Feb 15 18:17:54 2010
@@ -89,9 +89,9 @@
     DenseMatrix phi = new DenseMatrix(state.numTopics, docLength);
     
     // digamma is expensive, precompute
-    Vector digammaGamma = LDAInference.digamma(gamma);
+    Vector digammaGamma = digamma(gamma);
     // and log normalize:
-    double digammaSumGamma = LDAInference.digamma(gamma.zSum());
+    double digammaSumGamma = digamma(gamma.zSum());
     digammaGamma = digammaGamma.plus(-digammaSumGamma);
     
     Map<Integer,Integer> columnMap = new HashMap<Integer,Integer>();
@@ -100,7 +100,7 @@
     
     boolean converged = false;
     double oldLL = 1;
-    while (!converged && (iteration < LDAInference.MAX_ITER)) {
+    while (!converged && (iteration < MAX_ITER)) {
       nextGamma.assign(state.topicSmoothing); // nG := alpha, for all topics
       
       int mapping = 0;
@@ -126,14 +126,14 @@
       nextGamma = tempG;
       
       // digamma is expensive, precompute
-      digammaGamma = LDAInference.digamma(gamma);
+      digammaGamma = digamma(gamma);
       // and log normalize:
-      digammaSumGamma = LDAInference.digamma(gamma.zSum());
+      digammaSumGamma = digamma(gamma.zSum());
       digammaGamma = digammaGamma.plus(-digammaSumGamma);
       
       double ll = computeLikelihood(wordCounts, columnMap, phi, gamma, digammaGamma);
       assert !Double.isNaN(ll);
-      converged = (oldLL < 0) && ((oldLL - ll) / oldLL < LDAInference.E_STEP_CONVERGENCE);
+      converged = (oldLL < 0) && ((oldLL - ll) / oldLL < E_STEP_CONVERGENCE);
       
       oldLL = ll;
       iteration++;
@@ -210,7 +210,7 @@
     digammaGamma.assign(v, new BinaryFunction() {
       @Override
       public double apply(double unused, double g) {
-        return LDAInference.digamma(g);
+        return digamma(g);
       }
     });
     return digammaGamma;
@@ -242,14 +242,7 @@
     }
     
     double f = 1.0 / (x * x);
-    double t = f
-               * (-1 / 12.0 + f
-                              * (1 / 120.0 + f
-                                             * (-1 / 252.0 + f
-                                                             * (1 / 240.0 + f
-                                                                            * (-1 / 132.0 + f
-                                                                                            * (691 / 32760.0 + f
-                                                                                                               * (-1 / 12.0 + f * 3617.0 / 8160.0)))))));
+    double t = f * (-1 / 12.0 + f * (1 / 120.0 + f * (-1 / 252.0 + f * (1 / 240.0 + f * (-1 / 132.0 + f * (691 / 32760.0 + f * (-1 / 12.0 + f * 3617.0 / 8160.0)))))));
     return r + Math.log(x) - 0.5 / x + t;
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAReducer.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAReducer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/lda/LDAReducer.java Mon Feb 15 18:17:54 2010
@@ -26,8 +26,9 @@
 public class LDAReducer extends Reducer<IntPairWritable,DoubleWritable,IntPairWritable,DoubleWritable> {
   
   @Override
-  public void reduce(IntPairWritable topicWord, Iterable<DoubleWritable> values, Context context) throws java.io.IOException,
-                                                                                                 InterruptedException {
+  public void reduce(IntPairWritable topicWord,
+                     Iterable<DoubleWritable> values,
+                     Context context) throws java.io.IOException, InterruptedException {
     
     // sum likelihoods
     if (topicWord.getY() == LDADriver.LOG_LIKELIHOOD_KEY) {

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterer.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterer.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyClusterer.java Mon Feb 15 18:17:54 2010
@@ -23,20 +23,20 @@
   // the distance measure
   private DistanceMeasure measure;
   
-  public double getT1() {
-    return t1;
-  }
-  
-  public double getT2() {
-    return t2;
+  public MeanShiftCanopyClusterer(JobConf job) {
+    configure(job);
   }
   
   public MeanShiftCanopyClusterer(DistanceMeasure aMeasure, double aT1, double aT2, double aDelta) {
     config(aMeasure, aT1, aT2, aDelta);
   }
   
-  public MeanShiftCanopyClusterer(JobConf job) {
-    configure(job);
+  public double getT1() {
+    return t1;
+  }
+  
+  public double getT2() {
+    return t2;
   }
   
   /**
@@ -111,7 +111,8 @@
   }
   
   /** Emit the new canopy to the collector, keyed by the canopy's Id */
-  static void emitCanopy(MeanShiftCanopy canopy, OutputCollector<Text,WritableComparable<?>> collector) throws IOException {
+  static void emitCanopy(MeanShiftCanopy canopy,
+                         OutputCollector<Text,WritableComparable<?>> collector) throws IOException {
     String identifier = canopy.getIdentifier();
     collector.collect(new Text(identifier), new Text("new " + canopy.toString()));
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyDriver.java Mon Feb 15 18:17:54 2010
@@ -91,7 +91,7 @@
       MeanShiftCanopyDriver.runJob(input, output, output + MeanShiftCanopyConfigKeys.CONTROL_PATH_KEY,
         measureClassName, t1, t2, convergenceDelta);
     } catch (OptionException e) {
-      MeanShiftCanopyDriver.log.error("Exception parsing command line: ", e);
+      log.error("Exception parsing command line: ", e);
       CommandLineUtil.printHelp(group);
     }
   }
@@ -147,7 +147,7 @@
     try {
       JobClient.runJob(conf);
     } catch (IOException e) {
-      MeanShiftCanopyDriver.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
     }
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyJob.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/meanshift/MeanShiftCanopyJob.java Mon Feb 15 18:17:54 2010
@@ -87,9 +87,9 @@
       double t2 = Double.parseDouble(cmdLine.getValue(threshold2Opt).toString());
       double convergenceDelta = Double.parseDouble(cmdLine.getValue(convergenceDeltaOpt).toString());
       int maxIterations = Integer.parseInt(cmdLine.getValue(maxIterOpt).toString());
-      MeanShiftCanopyJob.runJob(input, output, measureClassName, t1, t2, convergenceDelta, maxIterations);
+      runJob(input, output, measureClassName, t1, t2, convergenceDelta, maxIterations);
     } catch (OptionException e) {
-      MeanShiftCanopyJob.log.error("Exception parsing command line: ", e);
+      log.error("Exception parsing command line: ", e);
       CommandLineUtil.printHelp(group);
     }
   }
@@ -132,10 +132,10 @@
     int iteration = 0;
     String clustersIn = input;
     while (!converged && (iteration < maxIterations)) {
-      MeanShiftCanopyJob.log.info("Iteration {}", iteration);
+      log.info("Iteration {}", iteration);
       // point the output to a new directory per iteration
       String clustersOut = output + "/canopies-" + iteration;
-      String controlOut = output + MeanShiftCanopyJob.CONTROL_CONVERGED;
+      String controlOut = output + CONTROL_CONVERGED;
       MeanShiftCanopyDriver.runJob(clustersIn, clustersOut, controlOut, measureClassName, t1, t2,
         convergenceDelta);
       converged = FileSystem.get(conf).exists(new Path(controlOut));

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/HadoopUtil.java Mon Feb 15 18:17:54 2010
@@ -38,17 +38,17 @@
     Path outPath = new Path(output);
     FileSystem fs = FileSystem.get(outPath.toUri(), conf);
     if (fs.exists(outPath)) {
-      HadoopUtil.log.warn("Deleting {}", outPath);
+      log.warn("Deleting {}", outPath);
       fs.delete(outPath, true);
     }
-    HadoopUtil.log.warn("Creating dir {}", outPath);
+    log.warn("Creating dir {}", outPath);
     fs.mkdirs(outPath);
   }
   
   public static void deletePath(String output, FileSystem fs) throws IOException {
     Path outPath = new Path(output);
     if (fs.exists(outPath)) {
-      HadoopUtil.log.warn("Deleting {}", outPath);
+      log.warn("Deleting {}", outPath);
       fs.delete(outPath, true);
     }
   }
@@ -56,14 +56,14 @@
   public static void deletePaths(Iterable<Path> paths, FileSystem fs) throws IOException {
     for (Path path : paths) {
       if (fs.exists(path)) {
-        HadoopUtil.log.warn("Deleting {}", path);
+        log.warn("Deleting {}", path);
         fs.delete(path, true);
       }
     }
   }
   
   public static void rename(Path from, Path to, FileSystem fs) throws IOException {
-    HadoopUtil.log.warn("Renaming " + from.toUri() + " to " + to.toUri());
+    log.warn("Renaming " + from.toUri() + " to " + to.toUri());
     fs.rename(from, to);
   }
 }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/IOUtils.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/IOUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/IOUtils.java Mon Feb 15 18:17:54 2010
@@ -43,7 +43,7 @@
       try {
         closeable.close();
       } catch (IOException ioe) {
-        IOUtils.log.warn("Unexpected exception while closing; continuing", ioe);
+        log.warn("Unexpected exception while closing; continuing", ioe);
       }
     }
   }
@@ -55,7 +55,7 @@
       try {
         closeable.close();
       } catch (SQLException sqle) {
-        IOUtils.log.warn("Unexpected exception while closing; continuing", sqle);
+        log.warn("Unexpected exception while closing; continuing", sqle);
       }
     }
   }
@@ -65,7 +65,7 @@
       try {
         closeable.close();
       } catch (SQLException sqle) {
-        IOUtils.log.warn("Unexpected exception while closing; continuing", sqle);
+        log.warn("Unexpected exception while closing; continuing", sqle);
       }
     }
   }
@@ -75,7 +75,7 @@
       try {
         closeable.close();
       } catch (SQLException sqle) {
-        IOUtils.log.warn("Unexpected exception while closing; continuing", sqle);
+        log.warn("Unexpected exception while closing; continuing", sqle);
       }
     }
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/Parameters.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/Parameters.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/Parameters.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/Parameters.java Mon Feb 15 18:17:54 2010
@@ -67,7 +67,7 @@
     try {
       return mapStringifier.toString(params);
     } catch (IOException e) {
-      Parameters.log.info("Encountered IOException while deserializing returning empty string", e);
+      log.info("Encountered IOException while deserializing returning empty string", e);
       return "";
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/RandomUtils.java Mon Feb 15 18:17:54 2010
@@ -44,16 +44,16 @@
   private RandomUtils() { }
   
   public static void useTestSeed() {
-    RandomUtils.testSeed = true;
+    testSeed = true;
   }
   
   public static Random getRandom() {
-    return RandomUtils.testSeed ? new MersenneTwisterRNG(RandomUtils.STANDARD_SEED)
+    return testSeed ? new MersenneTwisterRNG(STANDARD_SEED)
         : new MersenneTwisterRNG();
   }
   
   public static Random getRandom(long seed) {
-    return new MersenneTwisterRNG(RandomUtils.longSeedtoBytes(seed));
+    return new MersenneTwisterRNG(longSeedtoBytes(seed));
   }
   
   public static byte[] longSeedtoBytes(long seed) {
@@ -98,15 +98,15 @@
    * </p>
    */
   public static int nextTwinPrime(int n) {
-    if (n > RandomUtils.MAX_INT_SMALLER_TWIN_PRIME) {
+    if (n > MAX_INT_SMALLER_TWIN_PRIME) {
       throw new IllegalArgumentException();
     }
     if (n <= 3) {
       return 3;
     }
-    int next = RandomUtils.nextPrime(n);
-    while (RandomUtils.isNotPrime(next + 2)) {
-      next = RandomUtils.nextPrime(next + 4);
+    int next = nextPrime(n);
+    while (isNotPrime(next + 2)) {
+      next = nextPrime(next + 4);
     }
     return next + 2;
   }
@@ -123,7 +123,7 @@
     // Make sure the number is odd. Is this too clever?
     n |= 0x1;
     // There is no problem with overflow since Integer.MAX_INT is prime, as it happens
-    while (RandomUtils.isNotPrime(n)) {
+    while (isNotPrime(n)) {
       n += 2;
     }
     return n;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/distance/ManhattanDistanceMeasure.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/distance/ManhattanDistanceMeasure.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/distance/ManhattanDistanceMeasure.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/distance/ManhattanDistanceMeasure.java Mon Feb 15 18:17:54 2010
@@ -67,7 +67,7 @@
     // two
     while (iter.hasNext()) {
       Vector.Element e = iter.next();
-      result += Math.abs(v2.getQuick(e.index()) - v1.getQuick(e.index()));
+      result += Math.abs(e.get());
     }
     return result;
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/parameters/Parametered.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/parameters/Parametered.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/parameters/Parametered.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/common/parameters/Parametered.java Mon Feb 15 18:17:54 2010
@@ -87,8 +87,8 @@
     
     private static void configureParametersRecusivly(Parametered parametered, String prefix, JobConf jobConf) {
       for (Parameter<?> parameter : parametered.getParameters()) {
-        if (Parametered.log.isDebugEnabled()) {
-          Parametered.log.debug("Configuring {}{}", prefix, parameter.name());
+        if (log.isDebugEnabled()) {
+          log.debug("Configuring {}{}", prefix, parameter.name());
         }
         String name = prefix + parameter.name() + '.';
         parameter.createParameters(name, jobConf);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/Bagging.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/Bagging.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/Bagging.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/Bagging.java Mon Feb 15 18:17:54 2010
@@ -58,16 +58,16 @@
    *           if the data is not set
    */
   public Node build(int treeId, Random rng, PredictionCallback callback) {
-    Bagging.log.debug("Bagging...");
+    log.debug("Bagging...");
     Arrays.fill(sampled, false);
     Data bag = data.bagging(rng, sampled);
     
-    Bagging.log.debug("Building...");
+    log.debug("Building...");
     Node tree = treeBuilder.build(rng, bag);
     
     // predict the label for the out-of-bag elements
     if (callback != null) {
-      Bagging.log.debug("Oob error estimation");
+      log.debug("Oob error estimation");
       for (int index = 0; index < data.size(); index++) {
         if (sampled[index] == false) {
           int prediction = tree.classify(data.get(index));

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/DFUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/DFUtils.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/DFUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/DFUtils.java Mon Feb 15 18:17:54 2010
@@ -30,7 +30,7 @@
  * Utility class that contains various helper methods
  */
 public class DFUtils {
-  private DFUtils() {}
+  private DFUtils() { }
   
   /**
    * Writes an Node[] into a DataOutput

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/builder/DefaultTreeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/builder/DefaultTreeBuilder.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/builder/DefaultTreeBuilder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/builder/DefaultTreeBuilder.java Mon Feb 15 18:17:54 2010
@@ -95,7 +95,7 @@
     
     if (alreadySelected) {
       // attribute already selected
-      DefaultTreeBuilder.log.warn("attribute {} already selected in a parent node", best.attr);
+      log.warn("attribute {} already selected in a parent node", best.attr);
     }
     
     Node childNode;
@@ -175,7 +175,7 @@
     }
     
     if (nbNonSelected == 0) {
-      DefaultTreeBuilder.log.warn("All attributes are selected !");
+      log.warn("All attributes are selected !");
     }
     
     int[] result;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataConverter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataConverter.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataConverter.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataConverter.java Mon Feb 15 18:17:54 2010
@@ -67,7 +67,7 @@
       if (attr == dataset.getLabelId()) {
         label = dataset.labelCode(token);
         if (label == -1) {
-          DataConverter.log.error(String.format("label token: %s\ndataset.labels: %s", token, Arrays
+          log.error(String.format("label token: %s\ndataset.labels: %s", token, Arrays
               .toString(dataset.labels())));
           throw new IllegalStateException("Label value (" + token + ") not known");
         }
@@ -80,7 +80,7 @@
     }
     
     if (label == -1) {
-      DataConverter.log.error(String.format("Label not found, instance id : %d, \nstring : %s", id, string));
+      log.error(String.format("Label not found, instance id : %d, \nstring : %s", id, string));
       throw new IllegalStateException("Label not found!");
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataLoader.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataLoader.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataLoader.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataLoader.java Mon Feb 15 18:17:54 2010
@@ -46,7 +46,7 @@
   
   private static final Logger log = LoggerFactory.getLogger(DataLoader.class);
   
-  private DataLoader() {}
+  private DataLoader() { }
   
   /**
    * Converts a comma-separated String to a Vector.
@@ -63,7 +63,7 @@
   private static Instance parseString(int id, Attribute[] attrs, List<String>[] values, String string) {
     StringTokenizer tokenizer = new StringTokenizer(string, ", ");
     if (tokenizer.countTokens() != attrs.length) {
-      DataLoader.log.error("{}: {}", id, string);
+      log.error("{}: {}", id, string);
       throw new IllegalArgumentException("Wrong number of attributes in the string");
     }
     
@@ -146,14 +146,14 @@
     while (scanner.hasNextLine()) {
       String line = scanner.nextLine();
       if (line.isEmpty()) {
-        DataLoader.log.warn("{}: empty string", instances.size());
+        log.warn("{}: empty string", instances.size());
         continue;
       }
       
       Instance instance = converter.convert(instances.size(), line);
       if (instance == null) {
         // missing values found
-        DataLoader.log.warn("{}: missing values", instances.size());
+        log.warn("{}: missing values", instances.size());
         continue;
       }
       
@@ -175,14 +175,14 @@
     
     for (String line : data) {
       if (line.isEmpty()) {
-        DataLoader.log.warn("{}: empty string", instances.size());
+        log.warn("{}: empty string", instances.size());
         continue;
       }
       
       Instance instance = converter.convert(instances.size(), line);
       if (instance == null) {
         // missing values found
-        DataLoader.log.warn("{}: missing values", instances.size());
+        log.warn("{}: missing values", instances.size());
         continue;
       }
       

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataUtils.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DataUtils.java Mon Feb 15 18:17:54 2010
@@ -25,7 +25,7 @@
  * Helper methods that deals with data lists and arrays of values
  */
 public class DataUtils {
-  private DataUtils() {}
+  private DataUtils() { }
   
   /**
    * Computes the sum of the values

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/Dataset.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/Dataset.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/Dataset.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/Dataset.java Mon Feb 15 18:17:54 2010
@@ -135,7 +135,7 @@
     return ignored;
   }
   
-  private Dataset() {}
+  private Dataset() { }
   
   /**
    * Should only be called by a DataLoader

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DescriptorUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DescriptorUtils.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DescriptorUtils.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/data/DescriptorUtils.java Mon Feb 15 18:17:54 2010
@@ -28,7 +28,7 @@
  * Contains various methods that deal with descriptor strings
  */
 public class DescriptorUtils {
-  private DescriptorUtils() {}
+  private DescriptorUtils() { }
   
   /**
    * Parses a descriptor string and generates the corresponding array of Attributes

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/Builder.java Mon Feb 15 18:17:54 2010
@@ -282,14 +282,14 @@
     // put the dataset into the DistributedCache
     DistributedCache.addCacheFile(datasetPath.toUri(), job);
     
-    Builder.log.debug("Configuring the job...");
+    log.debug("Configuring the job...");
     configureJob(job, nbTrees, callback != null);
     
-    Builder.log.debug("Running the job...");
+    log.debug("Running the job...");
     runJob(job);
     
     if (Builder.isOutput(job)) {
-      Builder.log.debug("Parsing the output...");
+      log.debug("Parsing the output...");
       DecisionForest forest = parseOutput(job, callback);
       
       // delete the output path

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemInputFormat.java Mon Feb 15 18:17:54 2010
@@ -75,10 +75,10 @@
     int splitSize = nbTrees / numSplits;
     
     seed = Builder.getRandomSeed(conf);
-    isSingleSeed = InMemInputFormat.isSingleSeed(conf);
+    isSingleSeed = isSingleSeed(conf);
     
     if (rng != null && seed != null) {
-      InMemInputFormat.log.warn("getSplits() was called more than once and the 'seed' is set, "
+      log.warn("getSplits() was called more than once and the 'seed' is set, "
         + "this can lead to no-repeatable behavior");
     }
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemMapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/inmem/InMemMapper.java Mon Feb 15 18:17:54 2010
@@ -76,14 +76,14 @@
   public void configure(JobConf conf) {
     super.configure(conf);
     
-    InMemMapper.log.info("Loading the data...");
+    log.info("Loading the data...");
     try {
       data = InMemMapper.loadData(conf, getDataset());
     } catch (IOException e) {
       throw new IllegalStateException("Exception caught while loading the data: "
                                       + StringUtils.stringifyException(e));
     }
-    InMemMapper.log.info("Data loaded : {} instances", data.size());
+    log.info("Data loaded : {} instances", data.size());
     
     bagging = new Bagging(getTreeBuilder(), data);
   }
@@ -108,11 +108,11 @@
     
     initRandom(split);
     
-    InMemMapper.log.debug("Building...");
+    log.debug("Building...");
     Node tree = bagging.build(key.get(), rng, callback);
     
     if (!isNoOutput()) {
-      InMemMapper.log.debug("Outputing...");
+      log.debug("Outputing...");
       MapredOutput mrOut = new MapredOutput(tree, predictions);
       
       output.collect(key, mrOut);
@@ -122,7 +122,7 @@
   protected void initRandom(InMemInputSplit split) {
     if (rng == null) { // first execution of this mapper
       Long seed = split.getSeed();
-      InMemMapper.log.debug("Initialising rng with seed {}: ", seed);
+      log.debug("Initialising rng with seed {}: ", seed);
       
       if (seed == null) {
         rng = RandomUtils.getRandom();

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/PartialBuilder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/PartialBuilder.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/PartialBuilder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/PartialBuilder.java Mon Feb 15 18:17:54 2010
@@ -101,7 +101,7 @@
     // or the mappers won't be able to compute the right indexes
     String tracker = job.get("mapred.job.tracker", "local");
     if ("local".equals(tracker)) {
-      PartialBuilder.log.warn("Hadoop running in 'local' mode, only one map task will be launched");
+      log.warn("Hadoop running in 'local' mode, only one map task will be launched");
       job.setNumMapTasks(1);
     }
   }
@@ -113,11 +113,11 @@
     
     Path outputPath = getOutputPath(job);
     
-    PartialBuilder.log.info("Computing partitions' first ids...");
+    log.info("Computing partitions' first ids...");
     Step0Job step0 = new Step0Job(getOutputPath(job), getDataPath(), getDatasetPath());
     Step0Output[] partitions = step0.run(getConf());
     
-    PartialBuilder.log.info("Processing the output...");
+    log.info("Processing the output...");
     TreeID[] keys = new TreeID[numTrees];
     Node[] trees = new Node[numTrees];
     int[] firstIds = Step0Output.extractFirstIds(partitions);
@@ -125,9 +125,9 @@
     
     // call the second step in order to complete the oob predictions
     if ((callback != null) && (numMaps > 1) && PartialBuilder.isStep2(getConf())) {
-      PartialBuilder.log.info("*****************************");
-      PartialBuilder.log.info("Second Step");
-      PartialBuilder.log.info("*****************************");
+      log.info("*****************************");
+      log.info("Second Step");
+      log.info("*****************************");
       Step2Job step2 = new Step2Job(getOutputPath(job), getDataPath(), getDatasetPath(), partitions);
       
       step2.run(job, keys, trees, callback);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step1Mapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step1Mapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step1Mapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step1Mapper.java Mon Feb 15 18:17:54 2010
@@ -97,7 +97,7 @@
     converter = new DataConverter(getDataset());
     
     // prepare random-numders generator
-    Step1Mapper.log.debug("seed : {}", seed);
+    log.debug("seed : {}", seed);
     if (seed == null) {
       rng = RandomUtils.getRandom();
     } else {
@@ -119,9 +119,9 @@
       firstTreeId += Step1Mapper.nbTrees(numMapTasks, numTrees, p);
     }
     
-    Step1Mapper.log.debug("partition : {}", partition);
-    Step1Mapper.log.debug("nbTrees : {}", nbTrees);
-    Step1Mapper.log.debug("firstTreeId : {}", firstTreeId);
+    log.debug("partition : {}", partition);
+    log.debug("nbTrees : {}", nbTrees);
+    log.debug("firstTreeId : {}", firstTreeId);
   }
   
   /**
@@ -157,18 +157,18 @@
   @Override
   public void close() throws IOException {
     // prepare the data
-    Step1Mapper.log.debug("partition: {} numInstances: {}", partition, instances.size());
+    log.debug("partition: {} numInstances: {}", partition, instances.size());
     
     Data data = new Data(getDataset(), instances);
     Bagging bagging = new Bagging(getTreeBuilder(), data);
     
     TreeID key = new TreeID();
     
-    Step1Mapper.log.debug("Building {} trees", nbTrees);
+    log.debug("Building {} trees", nbTrees);
     SingleTreePredictions callback = null;
     int[] predictions = null;
     for (int treeId = 0; treeId < nbTrees; treeId++) {
-      Step1Mapper.log.debug("Building tree number: {}", treeId);
+      log.debug("Building tree number: {}", treeId);
       if (isOobEstimate() && !isNoOutput()) {
         callback = new SingleTreePredictions(data.size());
         predictions = callback.getPredictions();

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step2Mapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step2Mapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step2Mapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapred/partial/Step2Mapper.java Mon Feb 15 18:17:54 2010
@@ -108,7 +108,7 @@
       FileSystem fs = forestPath.getFileSystem(job);
       numInstances = InterResults.load(fs, forestPath, numMaps, numTrees, p, keys, trees);
       
-      Step2Mapper.log.debug("partition: {} numInstances: {}", p, numInstances);
+      log.debug("partition: {} numInstances: {}", p, numInstances);
     } catch (IOException e) {
       throw new IllegalStateException("Exception while loading the forest : ", e);
     }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/Builder.java Mon Feb 15 18:17:54 2010
@@ -92,7 +92,7 @@
     // or the mappers won't be able to compute the right indexes
     String tracker = conf.get("mapred.job.tracker", "local");
     if ("local".equals(tracker)) {
-      Builder.log.warn("Hadoop running in 'local' mode, only one map task will be launched");
+      log.warn("Hadoop running in 'local' mode, only one map task will be launched");
       return 1;
     }
     
@@ -325,17 +325,17 @@
     
     Job job = new Job(conf, "decision forest builder");
     
-    Builder.log.debug("Configuring the job...");
+    log.debug("Configuring the job...");
     configureJob(job, nbTrees, callback != null);
     
-    Builder.log.debug("Running the job...");
+    log.debug("Running the job...");
     if (!runJob(job)) {
-      Builder.log.error("Job failed!");
+      log.error("Job failed!");
       return null;
     }
     
     if (Builder.isOutput(conf)) {
-      Builder.log.debug("Parsing the output...");
+      log.debug("Parsing the output...");
       DecisionForest forest = parseOutput(job, callback);
       
       // delete the output path

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemInputFormat.java Mon Feb 15 18:17:54 2010
@@ -84,10 +84,10 @@
     int splitSize = nbTrees / numSplits;
     
     seed = Builder.getRandomSeed(conf);
-    isSingleSeed = InMemInputFormat.isSingleSeed(conf);
+    isSingleSeed = isSingleSeed(conf);
     
     if ((rng != null) && (seed != null)) {
-      InMemInputFormat.log.warn("getSplits() was called more than once and the 'seed' is set, "
+      log.warn("getSplits() was called more than once and the 'seed' is set, "
                                 + "this can lead to no-repeatable behavior");
     }
     
@@ -189,7 +189,7 @@
     
     private Long seed;
     
-    public InMemInputSplit() {}
+    public InMemInputSplit() { }
     
     public InMemInputSplit(int firstId, int nbTrees, Long seed) {
       this.firstId = firstId;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemMapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemMapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/inmem/InMemMapper.java Mon Feb 15 18:17:54 2010
@@ -73,9 +73,9 @@
     
     Configuration conf = context.getConfiguration();
     
-    InMemMapper.log.info("Loading the data...");
+    log.info("Loading the data...");
     data = InMemMapper.loadData(conf, getDataset());
-    InMemMapper.log.info("Data loaded : {} instances", data.size());
+    log.info("Data loaded : {} instances", data.size());
     
     bagging = new Bagging(getTreeBuilder(), data);
   }
@@ -98,11 +98,11 @@
     
     initRandom((InMemInputSplit) context.getInputSplit());
     
-    InMemMapper.log.debug("Building...");
+    log.debug("Building...");
     Node tree = bagging.build(key.get(), rng, callback);
     
     if (!isNoOutput()) {
-      InMemMapper.log.debug("Outputing...");
+      log.debug("Outputing...");
       MapredOutput mrOut = new MapredOutput(tree, predictions);
       
       context.write(key, mrOut);
@@ -112,7 +112,7 @@
   protected void initRandom(InMemInputSplit split) {
     if (rng == null) { // first execution of this mapper
       Long seed = split.getSeed();
-      InMemMapper.log.debug("Initialising rng with seed : {}", seed);
+      log.debug("Initialising rng with seed : {}", seed);
       
       if (seed == null) {
         rng = RandomUtils.getRandom();

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/InterResults.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/InterResults.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/InterResults.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/InterResults.java Mon Feb 15 18:17:54 2010
@@ -29,7 +29,7 @@
  * This class should not be needed outside of the partial package, so all its methods are protected.<br>
  */
 public class InterResults {
-  private InterResults() {}
+  private InterResults() { }
   
   /**
    * Load the trees and the keys returned from the first step

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/PartialBuilder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/PartialBuilder.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/PartialBuilder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/PartialBuilder.java Mon Feb 15 18:17:54 2010
@@ -112,11 +112,11 @@
     
     Path outputPath = getOutputPath(conf);
     
-    PartialBuilder.log.info("Computing partitions' first ids...");
+    log.info("Computing partitions' first ids...");
     Step0Job step0 = new Step0Job(getOutputPath(conf), getDataPath(), getDatasetPath());
     Step0Output[] partitions = step0.run(new Configuration(conf));
     
-    PartialBuilder.log.info("Processing the output...");
+    log.info("Processing the output...");
     TreeID[] keys = new TreeID[numTrees];
     Node[] trees = new Node[numTrees];
     int[] firstIds = Step0Output.extractFirstIds(partitions);
@@ -127,9 +127,9 @@
     
     // call the second step in order to complete the oob predictions
     if ((callback != null) && (numMaps > 1) && PartialBuilder.isStep2(conf)) {
-      PartialBuilder.log.info("*****************************");
-      PartialBuilder.log.info("Second Step");
-      PartialBuilder.log.info("*****************************");
+      log.info("*****************************");
+      log.info("Second Step");
+      log.info("*****************************");
       Step2Job step2 = new Step2Job(getOutputPath(conf), getDataPath(), getDatasetPath(), partitions);
       
       step2.run(new Configuration(conf), keys, trees, callback);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step0Job.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step0Job.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step0Job.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step0Job.java Mon Feb 15 18:17:54 2010
@@ -126,7 +126,7 @@
   protected Step0Output[] parseOutput(JobContext job) throws IOException {
     Configuration conf = job.getConfiguration();
     
-    Step0Job.log.info("mapred.map.tasks = {}", conf.getInt("mapred.map.tasks", -1));
+    log.info("mapred.map.tasks = {}", conf.getInt("mapred.map.tasks", -1));
     
     FileSystem fs = outputPath.getFileSystem(conf);
     

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step1Mapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step1Mapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step1Mapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step1Mapper.java Mon Feb 15 18:17:54 2010
@@ -91,7 +91,7 @@
     converter = new DataConverter(getDataset());
     
     // prepare random-numders generator
-    Step1Mapper.log.debug("seed : {}", seed);
+    log.debug("seed : {}", seed);
     if (seed == null) {
       rng = RandomUtils.getRandom();
     } else {
@@ -113,9 +113,9 @@
       firstTreeId += Step1Mapper.nbTrees(numMapTasks, numTrees, p);
     }
     
-    Step1Mapper.log.debug("partition : {}", partition);
-    Step1Mapper.log.debug("nbTrees : {}", nbTrees);
-    Step1Mapper.log.debug("firstTreeId : {}", firstTreeId);
+    log.debug("partition : {}", partition);
+    log.debug("nbTrees : {}", nbTrees);
+    log.debug("firstTreeId : {}", firstTreeId);
   }
   
   /**
@@ -147,18 +147,18 @@
   @Override
   protected void cleanup(Context context) throws IOException, InterruptedException {
     // prepare the data
-    Step1Mapper.log.debug("partition: {} numInstances: {}", partition, instances.size());
+    log.debug("partition: {} numInstances: {}", partition, instances.size());
     
     Data data = new Data(getDataset(), instances);
     Bagging bagging = new Bagging(getTreeBuilder(), data);
     
     TreeID key = new TreeID();
     
-    Step1Mapper.log.debug("Building {} trees", nbTrees);
+    log.debug("Building {} trees", nbTrees);
     SingleTreePredictions callback = null;
     int[] predictions = null;
     for (int treeId = 0; treeId < nbTrees; treeId++) {
-      Step1Mapper.log.debug("Building tree number : {}", treeId);
+      log.debug("Building tree number : {}", treeId);
       if (isOobEstimate() && !isNoOutput()) {
         callback = new SingleTreePredictions(data.size());
         predictions = callback.getPredictions();

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step2Mapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step2Mapper.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step2Mapper.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/Step2Mapper.java Mon Feb 15 18:17:54 2010
@@ -66,7 +66,7 @@
     // get the cached files' paths
     URI[] files = DistributedCache.getCacheFiles(conf);
     
-    Step2Mapper.log.info("DistributedCache.getCacheFiles(): {}", ArrayUtils.toString(files));
+    log.info("DistributedCache.getCacheFiles(): {}", ArrayUtils.toString(files));
     
     if ((files == null) || (files.length < 2)) {
       throw new IllegalArgumentException("missing paths from the DistributedCache");
@@ -92,7 +92,7 @@
     FileSystem fs = forestPath.getFileSystem(conf);
     int numInstances = InterResults.load(fs, forestPath, numMaps, numTrees, p, keys, trees);
     
-    Step2Mapper.log.debug("partition: {} numInstances: {}", p, numInstances);
+    log.debug("partition: {} numInstances: {}", p, numInstances);
     configure(p, dataset, keys, trees, numInstances);
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/TreeID.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/TreeID.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/TreeID.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/mapreduce/partial/TreeID.java Mon Feb 15 18:17:54 2010
@@ -39,7 +39,7 @@
   }
   
   public void set(int partition, int treeId) {
-    super.set((long) partition * TreeID.MAX_TREEID + treeId);
+    super.set((long) partition * MAX_TREEID + treeId);
   }
   
   /**
@@ -48,11 +48,11 @@
    * @return
    */
   public int partition() {
-    return (int) (get() / TreeID.MAX_TREEID);
+    return (int) (get() / MAX_TREEID);
   }
   
   public int treeId() {
-    return (int) (get() % TreeID.MAX_TREEID);
+    return (int) (get() % MAX_TREEID);
   }
   
   @Override

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/node/CategoricalNode.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/node/CategoricalNode.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/node/CategoricalNode.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/node/CategoricalNode.java Mon Feb 15 18:17:54 2010
@@ -33,7 +33,7 @@
   
   private Node[] childs;
   
-  public CategoricalNode() {}
+  public CategoricalNode() { }
   
   public CategoricalNode(int attr, double[] values, Node[] childs) {
     this.attr = attr;

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/ref/SequentialBuilder.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/ref/SequentialBuilder.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/ref/SequentialBuilder.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/ref/SequentialBuilder.java Mon Feb 15 18:17:54 2010
@@ -70,7 +70,7 @@
   private static void logProgress(float progress) {
     int percent = (int) (progress * 100);
     if (percent % 10 == 0) {
-      SequentialBuilder.log.info(String.format("Building %2d%%", percent));
+      log.info(String.format("Building %2d%%", percent));
     }
     
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Describe.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Describe.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Describe.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Describe.java Mon Feb 15 18:17:54 2010
@@ -49,7 +49,7 @@
   
   private static final Logger log = LoggerFactory.getLogger(Describe.class);
   
-  private Describe() {}
+  private Describe() { }
   
   public static void main(String[] args) throws IOException, DescriptorException {
     
@@ -88,28 +88,28 @@
       String descPath = cmdLine.getValue(descPathOpt).toString();
       List<String> descriptor = Describe.convert(cmdLine.getValues(descriptorOpt));
       
-      Describe.log.debug("Data path : {}", dataPath);
-      Describe.log.debug("Descriptor path : {}", descPath);
-      Describe.log.debug("Descriptor : {}", descriptor);
+      log.debug("Data path : {}", dataPath);
+      log.debug("Descriptor path : {}", descPath);
+      log.debug("Descriptor : {}", descriptor);
       
       Describe.runTool(dataPath, descriptor, descPath);
     } catch (OptionException e) {
-      Describe.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
       CommandLineUtil.printHelp(group);
     }
   }
   
   private static void runTool(String dataPath, List<String> description, String filePath) throws DescriptorException,
                                                                                          IOException {
-    Describe.log.info("Generating the descriptor...");
+    log.info("Generating the descriptor...");
     String descriptor = DescriptorUtils.generateDescriptor(description);
     
     Path fPath = Describe.validateOutput(filePath);
     
-    Describe.log.info("generating the dataset...");
+    log.info("generating the dataset...");
     Dataset dataset = Describe.generateDataset(descriptor, dataPath);
     
-    Describe.log.info("storing the dataset description");
+    log.info("storing the dataset description");
     Describe.storeWritable(new Configuration(), fPath, dataset);
   }
   

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Frequencies.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Frequencies.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Frequencies.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/Frequencies.java Mon Feb 15 18:17:54 2010
@@ -45,7 +45,7 @@
   
   private static final Logger log = LoggerFactory.getLogger(Frequencies.class);
   
-  private Frequencies() {}
+  private Frequencies() { }
   
   @Override
   public int run(String[] args) throws IOException, ClassNotFoundException, InterruptedException {
@@ -79,12 +79,12 @@
       String dataPath = cmdLine.getValue(dataOpt).toString();
       String datasetPath = cmdLine.getValue(datasetOpt).toString();
       
-      Frequencies.log.debug("Data path : {}", dataPath);
-      Frequencies.log.debug("Dataset path : {}", datasetPath);
+      log.debug("Data path : {}", dataPath);
+      log.debug("Dataset path : {}", datasetPath);
       
       runTool(dataPath, datasetPath);
     } catch (OptionException e) {
-      Frequencies.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
       CommandLineUtil.printHelp(group);
     }
     
@@ -101,7 +101,7 @@
     Path dataPath = new Path(data);
     Path datasetPath = new Path(dataset);
     
-    Frequencies.log.info("Computing the frequencies...");
+    log.info("Computing the frequencies...");
     FrequenciesJob job = new FrequenciesJob(new Path(workingDir, "output"), dataPath, datasetPath);
     
     int[][] counts = job.run(getConf());
@@ -114,9 +114,9 @@
     // }
     
     // outputing the frequencies
-    Frequencies.log.info("counts[partition][class]");
+    log.info("counts[partition][class]");
     for (int p = 0; p < numPartitions; p++) {
-      Frequencies.log.info(Arrays.toString(counts[p]));
+      log.info(Arrays.toString(counts[p]));
     }
   }
   
@@ -124,4 +124,4 @@
     ToolRunner.run(new Configuration(), new Frequencies(), args);
   }
   
-}
\ No newline at end of file
+}

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/FrequenciesJob.java Mon Feb 15 18:17:54 2010
@@ -29,9 +29,9 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -132,7 +132,7 @@
     Configuration conf = job.getConfiguration();
     
     int numMaps = conf.getInt("mapred.map.tasks", -1);
-    FrequenciesJob.log.info("mapred.map.tasks = {}", numMaps);
+    log.info("mapred.map.tasks = {}", numMaps);
     
     FileSystem fs = outputPath.getFileSystem(conf);
     
@@ -190,7 +190,7 @@
      */
     protected void setup(Dataset dataset) {
       converter = new DataConverter(dataset);
-      // log.info("labels: {}", Arrays.toString(dataset.labels()));
+      //log.info("labels: {}", Arrays.toString(dataset.labels()));
     }
     
     @Override
@@ -252,7 +252,7 @@
     /** counts[c] = num tuples from the partition with label == c */
     private int[] counts;
     
-    protected Frequencies() {}
+    protected Frequencies() { }
     
     protected Frequencies(long firstId, int[] counts) {
       this.firstId = firstId;
@@ -328,4 +328,4 @@
       return counts;
     }
   }
-}
\ No newline at end of file
+}

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/UDistrib.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/UDistrib.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/UDistrib.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/df/tools/UDistrib.java Mon Feb 15 18:17:54 2010
@@ -85,7 +85,7 @@
       
       UDistrib.runTool(data, dataset, output, numPartitions);
     } catch (OptionException e) {
-      UDistrib.log.warn(e.toString(), e);
+      log.warn(e.toString(), e);
       CommandLineUtil.printHelp(group);
     }
     
@@ -148,7 +148,7 @@
     int id = 0;
     while (scanner.hasNextLine()) {
       if (id % 1000 == 0) {
-        UDistrib.log.info(String.format("progress : %d / %d", id, nbInstances));
+        log.info(String.format("progress : %d / %d", id, nbInstances));
       }
       
       String line = scanner.nextLine();
@@ -179,7 +179,7 @@
     FileUtil.copyMerge(pfs, partsPath, fs, outputPath, true, conf, null);
     /*
      * FSDataOutputStream joined = fs.create(new Path(outputPath, "uniform.data")); for (int p = 0; p <
-     * numPartitions; p++) { log.info("Joining part : {}", p); FSDataInputStream partStream =
+     * numPartitions; p++) {log.info("Joining part : {}", p); FSDataInputStream partStream =
      * fs.open(partPaths[p]);
      * 
      * IOUtils.copyBytes(partStream, joined, conf, false);

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/FPGrowthDriver.java Mon Feb 15 18:17:54 2010
@@ -171,7 +171,7 @@
   }
   
   private static void runFPGrowth(Parameters params) throws IOException {
-    FPGrowthDriver.log.info("Starting Sequential FPGrowth");
+    log.info("Starting Sequential FPGrowth");
     int maxHeapSize = Integer.valueOf(params.get("maxHeapSize", "50"));
     int minSupport = Integer.valueOf(params.get("minSupport", "3"));
     
@@ -200,7 +200,7 @@
     
     List<Pair<String,TopKStringPatterns>> frequentPatterns = FPGrowth.readFrequentPattern(fs, conf, path);
     for (Pair<String,TopKStringPatterns> entry : frequentPatterns) {
-      FPGrowthDriver.log.info("Dumping Patterns for Feature: {} \n{}", entry.getFirst(), entry.getSecond()
+      log.info("Dumping Patterns for Feature: {} \n{}", entry.getFirst(), entry.getSecond()
           .toString());
     }
   }

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java?rev=910282&r1=910281&r2=910282&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/fpm/pfpgrowth/PFPGrowth.java Mon Feb 15 18:17:54 2010
@@ -269,7 +269,7 @@
       i++;
     }
     
-    PFPGrowth.log.info("No of Features: {}", fList.size());
+    log.info("No of Features: {}", fList.size());
     
     params.set("gList", PFPGrowth.serializeMap(gList, conf));
     params.set("fList", PFPGrowth.serializeList(fList, conf));