You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by gs...@apache.org on 2008/08/15 16:09:37 UTC

svn commit: r686221 - in /lucene/mahout/trunk: core/src/main/java/org/apache/mahout/ga/watchmaker/ core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/ core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/ core/src/test/examples/org/a...

Author: gsingers
Date: Fri Aug 15 07:09:36 2008
New Revision: 686221

URL: http://svn.apache.org/viewvc?rev=686221&view=rev
Log:
MAHOUT-56 more watchmaker updates and examples, going to move core/src/test/examples after committing, just to make the svn move easier

Added:
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosToolTest.java   (with props)
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtilsTest.java   (with props)
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java   (with props)
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapperTest.java   (with props)
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java   (with props)
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosTool.java   (with props)
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtils.java   (with props)
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/Descriptors.java   (with props)
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java   (with props)
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapper.java   (with props)
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java   (with props)
Removed:
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/ga/watchmaker/StringUtilsTest.java
Modified:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/DataLineTest.java
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluatorTest.java
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapperTest.java
    lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/utils/RandomRule.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDFitnessEvaluator.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDGA.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/FileInfoParser.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluator.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapper.java

Modified: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java (original)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/ga/watchmaker/MahoutEvaluator.java Fri Aug 15 07:09:36 2008
@@ -1,4 +1,5 @@
 package org.apache.mahout.ga.watchmaker;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -23,6 +24,7 @@
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.FloatWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -80,10 +82,13 @@
       throws IOException {
     Path inpath = new Path(fs.getWorkingDirectory(), "input");
 
-    if (!fs.exists(inpath)) {
-      fs.mkdirs(inpath);
+    // Delete the input if it already exists
+    if (fs.exists(inpath)) {
+      FileUtil.fullyDelete(fs, inpath);
     }
 
+    fs.mkdirs(inpath);
+
     storePopulation(fs, new Path(inpath, "population"), population);
 
     return inpath;

Modified: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/DataLineTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/DataLineTest.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/DataLineTest.java (original)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/DataLineTest.java Fri Aug 15 07:09:36 2008
@@ -16,18 +16,21 @@
   public void testSet() throws Exception {
     FileSystem fs = FileSystem.get(new Configuration());
     Path inpath = new Path("build/examples-test-classes/wdbc");
-    DataSet.initialize(FileInfoParser.parseFile(fs, inpath));
+    DataSet dataset = FileInfoParser.parseFile(fs, inpath);
+    DataSet.initialize(dataset);
     
     DataLine dl = new DataLine();
     
+    int labelpos = dataset.getLabelIndex();
+    
     dl.set(datalines[0]);
-    assertEquals(1, dl.getLabel());
+    assertEquals(dataset.valueIndex(labelpos, "M"), dl.getLabel());
     
     dl.set(datalines[1]);
-    assertEquals(0, dl.getLabel());
+    assertEquals(dataset.valueIndex(labelpos, "B"), dl.getLabel());
     
     dl.set(datalines[2]);
-    assertEquals(1, dl.getLabel());
+    assertEquals(dataset.valueIndex(labelpos, "M"), dl.getLabel());
   }
 
 }

Modified: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluatorTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluatorTest.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluatorTest.java (original)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluatorTest.java Fri Aug 15 07:09:36 2008
@@ -19,11 +19,12 @@
   public void testEvaluate() throws Exception {
     int nbrules = 100;
     Random rng = new MersenneTwisterRNG();
+    int target = 1;
 
     // random rules
     List<Rule> rules = new ArrayList<Rule>();
     for (int index = 0; index < nbrules; index++) {
-      rules.add(new RandomRule(index, rng));
+      rules.add(new RandomRule(index, target, rng));
     }
 
     // dataset
@@ -32,7 +33,7 @@
 
     // evaluate the rules
     List<CDFitness> results = new ArrayList<CDFitness>();
-    CDMahoutEvaluator.evaluate(rules, input, results);
+    CDMahoutEvaluator.evaluate(rules, target, input, results);
 
     // check the results
     for (int index = 0; index < nbrules; index++) {

Modified: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapperTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapperTest.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapperTest.java (original)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapperTest.java Fri Aug 15 07:09:36 2008
@@ -77,10 +77,10 @@
 
   public void testEvaluate() {
     // test the evaluation
-    assertEquals(TP, CDMapper.evaluate(1, 1));
-    assertEquals(FP, CDMapper.evaluate(1, 0));
-    assertEquals(TN, CDMapper.evaluate(0, 0));
-    assertEquals(FN, CDMapper.evaluate(0, 1));
+    assertEquals(TP, CDMapper.evaluate(1, 1, 1));
+    assertEquals(FP, CDMapper.evaluate(1, 1, 0));
+    assertEquals(TN, CDMapper.evaluate(1, 0, 0));
+    assertEquals(FN, CDMapper.evaluate(1, 0, 1));
   }
 
   public void testMap() throws Exception {
@@ -90,7 +90,7 @@
     // create and configure the mapper
     CDMapper mapper = new CDMapper();
     List<Rule> rules = Arrays.asList(rule, rule, rule, rule);
-    mapper.configure(rules);
+    mapper.configure(rules, 1);
 
     // test the mapper
     DummyOutputCollector<LongWritable, CDFitness> collector = new DummyOutputCollector<LongWritable, CDFitness>();

Added: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosToolTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosToolTest.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosToolTest.java (added)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosToolTest.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,215 @@
+package org.apache.mahout.ga.watchmaker.cd.tool;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.mahout.ga.watchmaker.cd.tool.DescriptionUtils.Range;
+import org.uncommons.maths.random.MersenneTwisterRNG;
+
+public class CDInfosToolTest extends TestCase {
+
+  Random rng;
+
+  @Override
+  protected void setUp() throws Exception {
+    rng = new MersenneTwisterRNG();
+  }
+
+  private Descriptors randomDescriptors(int nbattributes, float numRate,
+      float catRate) {
+    char[] descriptors = new char[nbattributes];
+    float rnd;
+    for (int index = 0; index < nbattributes; index++) {
+      rnd = rng.nextFloat();
+      if (rnd < numRate) {
+        // numerical attribute
+        descriptors[index] = 'N';
+      } else if (rnd < (numRate + catRate)) {
+        // categorical attribute
+        descriptors[index] = 'C';
+      } else {
+        // ignored attribute
+        descriptors[index] = 'I';
+      }
+    }
+
+    return new Descriptors(descriptors);
+  }
+
+  private Object[][] randomDescriptions(Descriptors descriptors) {
+    int nbattrs = descriptors.size();
+    Object[][] descriptions = new Object[nbattrs][];
+    double min, max;
+
+    for (int index = 0; index < nbattrs; index++) {
+      if (descriptors.isNumerical(index)) {
+        // numerical attribute
+        descriptions[index] = new Object[2];
+        min = rng.nextDouble() * (Float.MAX_VALUE - Float.MIN_VALUE)
+            + Float.MIN_VALUE;
+        max = rng.nextDouble() * (Float.MAX_VALUE - min) + min;
+
+        descriptions[index][0] = (float) min;
+        descriptions[index][1] = (float) max;
+      } else if (descriptors.isNominal(index)) {
+        // categorical attribute
+        int nbvalues = rng.nextInt(50) + 1;
+        descriptions[index] = new Object[nbvalues];
+        for (int vindex = 0; vindex < nbvalues; vindex++) {
+          descriptions[index][vindex] = "val_" + index + "_" + vindex;
+        }
+      }
+    }
+
+    return descriptions;
+  }
+
+  private void randomDataset(FileSystem fs, Path input, Descriptors descriptors,
+      Object[][] descriptions) throws IOException {
+    int nbfiles = rng.nextInt(20) + 1;
+    FSDataOutputStream out;
+    BufferedWriter writer;
+
+    for (int floop = 0; floop < nbfiles; floop++) {
+      out = fs.create(new Path(input, "file." + floop));
+      writer = new BufferedWriter(new OutputStreamWriter(out));
+
+      int nblines = rng.nextInt(200) + 1;
+      for (int line = 0; line < nblines; line++) {
+        writer.write(randomLine(descriptors, descriptions));
+        writer.newLine();
+      }
+
+      writer.close();
+    }
+  }
+
+  private String randomLine(Descriptors descriptors, Object[][] descriptions) {
+    StringBuffer buffer = new StringBuffer();
+
+    for (int index = 0; index < descriptors.size(); index++) {
+      if (descriptors.isNumerical(index)) {
+        // numerical attribute
+        float min = (Float) descriptions[index][0];
+        float max = (Float) descriptions[index][1];
+        float value = rng.nextFloat() * (max - min) + min;
+
+        buffer.append(value);
+      } else if (descriptors.isNominal(index)) {
+        // categorical attribute
+        int nbvalues = descriptions[index].length;
+        int vindex = rng.nextInt(nbvalues);
+
+        buffer.append(descriptions[index][vindex]);
+      } else {
+        // ignored attribute (any value is correct)
+        buffer.append("I");
+      }
+
+      if (index < descriptors.size() - 1) {
+        buffer.append(",");
+      }
+    }
+
+    return buffer.toString();
+  }
+
+  private int nbNonIgnored(Descriptors descriptors) {
+    int nbattrs = 0;
+    for (int index = 0; index < descriptors.size(); index++) {
+      if (!descriptors.isIgnored(index))
+        nbattrs++;
+    }
+    
+    return nbattrs;
+  }
+
+  public void testGatherInfos() throws Exception {
+    int maxattr = 100; // max number of attributes
+    int nbattrs = rng.nextInt(maxattr) + 1;
+
+    // random descriptors
+    float numRate = rng.nextFloat();
+    float catRate = rng.nextFloat() * (1f - numRate);
+    Descriptors descriptors = randomDescriptors(nbattrs, numRate, catRate);
+
+    // random descriptions
+    Object[][] descriptions = randomDescriptions(descriptors);
+
+    // random dataset
+    FileSystem fs = FileSystem.get(new Configuration());
+    Path inpath = new Path("input");
+    if (fs.exists(inpath)) {
+      FileUtil.fullyDelete(fs, inpath);
+    }
+
+    randomDataset(fs, inpath, descriptors, descriptions);
+
+    // Start the tool
+    List<String> result = new ArrayList<String>();
+    int rindex=0;
+    CDInfosTool.gatherInfos(descriptors, inpath, result);
+
+    // check the results
+    Collection<String> target = new ArrayList<String>();
+
+    assertEquals(nbNonIgnored(descriptors), result.size());
+    for (int index = 0; index < nbattrs; index++) {
+      if (descriptors.isIgnored(index)) {
+        continue;
+      } 
+
+      String description = result.get(rindex++);
+
+      if (descriptors.isNumerical(index)) {
+        // numerical attribute
+        float min = (Float) descriptions[index][0];
+        float max = (Float) descriptions[index][1];
+        Range range = DescriptionUtils.extractNumericalRange(description);
+
+        assertTrue("bad min value for attribute (" + index + ")",
+            min <= range.min);
+        assertTrue("bad max value for attribute (" + index + ")",
+            max >= range.max);
+      } else if (descriptors.isNominal(index)) {
+        // categorical attribute
+        Object[] values = descriptions[index];
+        target.clear();
+        DescriptionUtils.extractNominalValues(description, target);
+
+        assertEquals(values.length, target.size());
+        assertTrue(target.containsAll(Arrays.asList(values)));
+      }
+    }
+  }
+
+}

Propchange: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosToolTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtilsTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtilsTest.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtilsTest.java (added)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtilsTest.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,65 @@
+package org.apache.mahout.ga.watchmaker.cd.tool;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.mahout.ga.watchmaker.cd.tool.DescriptionUtils.Range;
+
+import junit.framework.TestCase;
+
+public class DescriptionUtilsTest extends TestCase {
+
+  public void testCreateNominalDescription() {
+    List<String> values = Arrays.asList("val1", "val2", "val3");
+
+    String description = DescriptionUtils.createNominalDescription(values);
+
+    assertEquals("val1,val2,val3", description);
+  }
+
+  public void testCreateNumericalDescription() {
+    String description = DescriptionUtils.createNumericalDescription(-5.1f,
+        12.32f);
+
+    assertEquals("-5.1,12.32", description);
+  }
+
+  public void testExtractNominalValues() {
+    String description = "val1,val2,val3";
+    List<String> target = new ArrayList<String>();
+
+    DescriptionUtils.extractNominalValues(description, target);
+
+    assertEquals(3, target.size());
+    assertTrue("'val1 not found'", target.contains("val1"));
+    assertTrue("'val2 not found'", target.contains("val2"));
+    assertTrue("'val3 not found'", target.contains("val3"));
+  }
+  
+  public void testExtractNumericalRange() {
+    String description = "-2.06,12.32";
+    
+    Range range = DescriptionUtils.extractNumericalRange(description);
+    
+    assertEquals(-2.06f, range.min);
+    assertEquals(12.32f, range.max);
+  }
+
+}

Propchange: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtilsTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java (added)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,85 @@
+package org.apache.mahout.ga.watchmaker.cd.tool;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+
+public class ToolCombinerTest extends TestCase {
+
+  public void testCreateDescriptionNumerical() throws Exception {
+    ToolCombiner combiner = new ToolCombiner();
+
+    char[] descriptors = { 'I', 'N', 'C' };
+    combiner.configure(descriptors);
+
+    List<Text> values = asList("0", "10", "-32", "0.5", "-30");
+    String descriptor = combiner.createDescription(1, values.iterator());
+
+    assertEquals("-32.0,10.0", descriptor);
+  }
+
+  public void testCreateDescriptionIgnored() throws Exception {
+    ToolCombiner combiner = new ToolCombiner();
+
+    char[] descriptors = { 'I', 'N', 'C' };
+    combiner.configure(descriptors);
+
+    try {
+      combiner.createDescription(0, null);
+      fail("Should throw a RuntimeException");
+    } catch (RuntimeException e) {
+
+    }
+  }
+
+  public void testCreateDescriptionNominal() throws Exception {
+    ToolCombiner combiner = new ToolCombiner();
+
+    char[] descriptors = { 'I', 'N', 'C' };
+    combiner.configure(descriptors);
+
+    List<Text> values = asList("val1", "val2", "val1", "val3", "val2");
+    String descriptor = combiner.createDescription(2, values.iterator());
+
+    StringTokenizer tokenizer = new StringTokenizer(descriptor, ",");
+    int nbvalues = 0;
+    while (tokenizer.hasMoreTokens()) {
+      String value = tokenizer.nextToken().trim();
+      if (!"val1".equals(value) && !"val2".equals(value)
+          && !"val3".equals(value)) {
+        fail("Incorrect value : " + value);
+      }
+      nbvalues++;
+    }
+    assertEquals(3, nbvalues);
+  }
+
+  List<Text> asList(String... strings) {
+    List<Text> values = new ArrayList<Text>();
+
+    for (String value : strings) {
+      values.add(new Text(value));
+    }
+    return values;
+  }
+}

Propchange: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombinerTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapperTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapperTest.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapperTest.java (added)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapperTest.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,79 @@
+package org.apache.mahout.ga.watchmaker.cd.tool;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.mahout.utils.DummyOutputCollector;
+
+public class ToolMapperTest extends TestCase {
+
+  public void testExtractAttributes() throws Exception {
+    LongWritable key = new LongWritable();
+    Text value = new Text();
+    DummyOutputCollector<LongWritable, Text> output = new DummyOutputCollector<LongWritable, Text>();
+
+    ToolMapper mapper = new ToolMapper();
+
+    // no attribute is ignored
+    String dataline = "A1, A2, A3, A4, A5, A6";
+    char[] descriptors = { 'N', 'N', 'C', 'C', 'N', 'N' };
+
+    mapper.configure(descriptors);
+    value.set(dataline);
+    mapper.map(key, value, output, null);
+
+    for (int index = 0; index < 6; index++) {
+      List<Text> values = output.getValue(String.valueOf(index));
+      assertEquals("should extract one value per attribute", 1, values.size());
+      assertEquals("Bad extracted value", "A" + (index + 1), values.get(0)
+          .toString());
+    }
+  }
+
+  public void testExtractIgnoredAttributes() throws Exception {
+    LongWritable key = new LongWritable();
+    Text value = new Text();
+    DummyOutputCollector<LongWritable, Text> output = new DummyOutputCollector<LongWritable, Text>();
+
+    ToolMapper mapper = new ToolMapper();
+
+    // no attribute is ignored
+    String dataline = "A1, I, A3, I, I, A6";
+    char[] descriptors = { 'N', 'I', 'C', 'I', 'I', 'N' };
+
+    mapper.configure(descriptors);
+    value.set(dataline);
+    mapper.map(key, value, output, null);
+
+    for (int index = 0; index < 6; index++) {
+      List<Text> values = output.getValue(String.valueOf(index));
+      if (index == 1 || index == 3 || index == 4) {
+        // this attribute should be ignored
+        assertNull("Attribute (" + index + ") should be ignored", values);
+      } else {
+        assertEquals("should extract one value per attribute", 1, values.size());
+        assertEquals("Bad extracted value", "A" + (index + 1), values.get(0)
+            .toString());
+      }
+    }
+  }
+}

Propchange: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapperTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java (added)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,84 @@
+package org.apache.mahout.ga.watchmaker.cd.tool;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+
+public class ToolReducerTest extends TestCase {
+
+  public void testCreateDescriptionNumerical() throws Exception {
+    ToolReducer reducer = new ToolReducer();
+
+    char[] descriptors = { 'I', 'N', 'C' };
+    reducer.configure(descriptors);
+
+    List<Text> values = asList("0,5", "-2,12", "-32,3",
+        "0.5,25", "-30,20");
+    String descriptor = reducer.numericDescription(values.iterator());
+
+    assertEquals("-32.0,25.0", descriptor);
+  }
+
+  public void testCreateDescriptionIgnored() throws Exception {
+    ToolReducer reducer = new ToolReducer();
+
+    char[] descriptors = { 'I', 'N', 'C' };
+    reducer.configure(descriptors);
+
+    try {
+      reducer.combineDescriptions(0, null);
+      fail("Should throw a RuntimeException");
+    } catch (RuntimeException e) {
+
+    }
+  }
+
+  public void testCreateDescriptionNominal() throws Exception {
+    ToolReducer reducer = new ToolReducer();
+
+    char[] descriptors = { 'I', 'N', 'C' };
+    reducer.configure(descriptors);
+
+    List<Text> values = asList("val1,val2", "val2,val3", "val1,val3", "val3",
+        "val2,val4");
+    List<String> expected = Arrays.asList("val1", "val2", "val3", "val4");
+
+    String description = reducer.nominalDescription(values.iterator());
+
+    Collection<String> actual = new ArrayList<String>();
+    DescriptionUtils.extractNominalValues(description, actual);
+
+    assertEquals(expected.size(), actual.size());
+    assertTrue(expected.containsAll(actual));
+  }
+
+  List<Text> asList(String... strings) {
+    List<Text> values = new ArrayList<Text>();
+
+    for (String value : strings) {
+      values.add(new Text(value));
+    }
+    return values;
+  }
+}

Propchange: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducerTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/utils/RandomRule.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/utils/RandomRule.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/utils/RandomRule.java (original)
+++ lucene/mahout/trunk/core/src/test/examples/org/apache/mahout/ga/watchmaker/cd/utils/RandomRule.java Fri Aug 15 07:09:36 2008
@@ -9,12 +9,15 @@
 
 public class RandomRule implements Rule {
 
-  private Random rng;
+  private final Random rng;
 
-  private int ruleid;
+  private final int ruleid;
+  
+  private final int target;
 
-  public RandomRule(int ruleid, Random rng) {
+  public RandomRule(int ruleid, int target, Random rng) {
     this.ruleid = ruleid;
+    this.target = target;
     this.rng = rng;
   }
 
@@ -22,7 +25,7 @@
     int label = dl.getLabel();
     int prediction = rng.nextInt(2);
 
-    CDFitness fitness = CDMapper.evaluate(prediction, label);
+    CDFitness fitness = CDMapper.evaluate(target, prediction, label);
     RandomRuleResults.addResult(ruleid, fitness);
 
     return prediction;

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDFitnessEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDFitnessEvaluator.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDFitnessEvaluator.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDFitnessEvaluator.java Fri Aug 15 07:09:36 2008
@@ -37,14 +37,17 @@
   private final DatasetSplit split;
 
   private final List<CDFitness> evals = new ArrayList<CDFitness>();
+  
+  private final int target;
 
   /**
    * 
    * @param dataset dataset path
    * @param split
    */
-  public CDFitnessEvaluator(String dataset, DatasetSplit split) {
+  public CDFitnessEvaluator(String dataset, int target, DatasetSplit split) {
     this.dataset = new Path(dataset);
+    this.target = target;
     this.split = split;
   }
 
@@ -59,7 +62,7 @@
     evals.clear();
 
     try {
-      CDMahoutEvaluator.evaluate(population, dataset, evals, split);
+      CDMahoutEvaluator.evaluate(population, target, dataset, evals, split);
     } catch (IOException e) {
       throw new RuntimeException("Exception while evaluating the population", e);
     }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDGA.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDGA.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDGA.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/CDGA.java Fri Aug 15 07:09:36 2008
@@ -69,6 +69,7 @@
    */
   public static void main(String[] args) throws IOException {
     String dataset = "build/classes/wdbc";
+    int target = 1;
     double threshold = 0.5;
     int crosspnts = 1;
     double mutrate = 0.1;
@@ -89,11 +90,11 @@
     }
 
     runJob(dataset, threshold, crosspnts, mutrate, mutrange, mutprec, popSize,
-        genCount);
+        genCount, target);
   }
 
   private static void runJob(String dataset, double threshold, int crosspnts,
-      double mutrate, double mutrange, int mutprec, int popSize, int genCount)
+                             double mutrate, double mutrange, int mutprec, int popSize, int genCount, int target)
       throws IOException {
     Path inpath = new Path(dataset);
     CDMahoutEvaluator.InitializeDataSet(inpath);
@@ -112,7 +113,7 @@
 
     // Fitness Evaluator (defaults to training)
     STFitnessEvaluator<? super Rule> evaluator = new CDFitnessEvaluator(
-        dataset, split);
+        dataset, target, split);
     // Selection Strategy
     SelectionStrategy selection = new RouletteWheelSelection();
 
@@ -127,16 +128,32 @@
 
     // evolve the rules over the training set
     Rule solution = engine.evolve(popSize, 1, new GenerationCount(genCount));
-    
+
     // fitness over the training set
-    CDFitness bestTrainFit = CDMahoutEvaluator.evaluate(solution, inpath, split);
+    CDFitness bestTrainFit = CDMahoutEvaluator.evaluate(solution, target,
+        inpath, split);
 
     // fitness over the testing set
     split.setTraining(false);
-    CDFitness bestTestFit = CDMahoutEvaluator.evaluate(solution, inpath, split);
+    CDFitness bestTestFit = CDMahoutEvaluator.evaluate(solution, target,
+        inpath, split);
 
     // evaluate the solution over the testing set
     System.out.println("Best solution fitness (train set) : " + bestTrainFit);
     System.out.println("Best solution fitness (test set) : " + bestTestFit);
   }
+
+  static void printElapsedTime(long milli) {
+    long seconds = milli / 1000;
+    milli = milli % 1000;
+
+    long minutes = seconds / 60;
+    seconds = seconds % 60;
+
+    long hours = minutes / 60;
+    minutes = minutes % 60;
+
+    System.out.println("Elapsed time (Hours:minutes:seconds:milli) : " + hours
+        + ":" + minutes + ":" + seconds + ":" + milli);
+  }
 }

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/FileInfoParser.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/FileInfoParser.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/FileInfoParser.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/FileInfoParser.java Fri Aug 15 07:09:36 2008
@@ -39,7 +39,7 @@
  * if the attribute is ignored</li>
  * <li><code>LABEL, val1, val2, ...</code><br>
  * if the attribute is the label, and its possible values</li>
- * <li><code>NOMINAL, val1, val2, ...</code><br>
+ * <li><code>CATEGORICAL, val1, val2, ...</code><br>
  * if the attribute is nominal, and its possible values</li>
  * <li><code>NUMERICAL, min, max</code><br>
  * if the attribute is numerical, and its min and max values</li>
@@ -47,13 +47,13 @@
  */
 public class FileInfoParser {
 
-  private static final String IGNORED_TOKEN = "IGNORED";
+  public static final String IGNORED_TOKEN = "IGNORED";
 
-  private static final String LABEL_TOKEN = "LABEL";
+  public  static final String LABEL_TOKEN = "LABEL";
 
-  private static final String NOMINAL_TOKEN = "NOMINAL";
+  public  static final String NOMINAL_TOKEN = "CATEGORICAL";
 
-  private static final String NUMERICAL_TOKEN = "NUMERICAL";
+  public static final String NUMERICAL_TOKEN = "NUMERICAL";
 
   /**
    * Initializes a dataset using an info file.
@@ -105,16 +105,16 @@
   }
 
   /**
-   * Prepares the path fot he info file corresponding to the input path.
+   * Prepares the path for the info file corresponding to the input path.
    * 
    * @param fs file system
    * @param inpath
    * @return
    * @throws IOException
    */
-  private static Path getInfoFile(FileSystem fs, Path inpath)
+  public static Path getInfoFile(FileSystem fs, Path inpath)
       throws IOException {
-    assert inpath != null : "null file parameter";
+    assert inpath != null : "null inpath parameter";
     if (!fs.exists(inpath))
       throw new RuntimeException("Input path does not exist");
     if (!fs.getFileStatus(inpath).isDir())

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluator.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluator.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMahoutEvaluator.java Fri Aug 15 07:09:36 2008
@@ -51,15 +51,17 @@
    * The input path contains the dataset
    * 
    * @param rules classification rules to evaluate
+   * @param target label value to evaluate the rules for
    * @param inpath input path (the dataset)
-   * @param evaluations <code>List&lt;Fitness&gt;</code> that contains the
+   * @param evaluations <code>List&lt;CDFitness&gt;</code> that contains the
    *        evaluated fitness for each candidate from the input population,
    *        sorted in the same order as the candidates.
    * @param split DatasetSplit used to separate training and testing input
    * @throws IOException
    */
-  public static void evaluate(List<? extends Rule> rules, Path inpath,
-      List<CDFitness> evaluations, DatasetSplit split) throws IOException {
+  public static void evaluate(List<? extends Rule> rules, int target,
+      Path inpath, List<CDFitness> evaluations, DatasetSplit split)
+      throws IOException {
     JobConf conf = new JobConf(CDMahoutEvaluator.class);
     FileSystem fs = FileSystem.get(conf);
 
@@ -69,12 +71,18 @@
 
     Path outpath = OutputUtils.prepareOutput(fs);
 
-    configureJob(conf, rules, inpath, outpath, split);
+    configureJob(conf, rules, target, inpath, outpath, split);
     JobClient.runJob(conf);
 
     importEvaluations(fs, conf, outpath, evaluations);
   }
 
+  /**
+   * Initializes the dataset
+   * 
+   * @param inpath input path (the dataset)
+   * @throws IOException
+   */
   public static void InitializeDataSet(Path inpath) throws IOException {
     JobConf conf = new JobConf(CDMahoutEvaluator.class);
     FileSystem fs = FileSystem.get(conf);
@@ -86,17 +94,18 @@
   /**
    * Evaluate a single rule.
    * 
-   * @param rule
-   * @param inpath
-   * @param split
+   * @param rule classification rule to evaluate
+   * @param target label value to evaluate the rules for
+   * @param inpath input path (the dataset)
+   * @param split DatasetSplit used to separate training and testing input
    * @return the evaluation
    * @throws IOException
    */
-  public static CDFitness evaluate(Rule rule, Path inpath, DatasetSplit split)
-      throws IOException {
+  public static CDFitness evaluate(Rule rule, int target, Path inpath,
+      DatasetSplit split) throws IOException {
     List<CDFitness> evals = new ArrayList<CDFitness>();
 
-    evaluate(Arrays.asList(rule), inpath, evals, split);
+    evaluate(Arrays.asList(rule), target, inpath, evals, split);
 
     return evals.get(0);
   }
@@ -104,25 +113,31 @@
   /**
    * Use all the dataset for training.
    * 
-   * @param rules
-   * @param inpath
-   * @param evaluations
+   * @param rules classification rules to evaluate
+   * @param target label value to evaluate the rules for
+   * @param inpath input path (the dataset)
+   * @param evaluations <code>List&lt;CDFitness&gt;</code> that contains the
+   *        evaluated fitness for each candidate from the input population,
+   *        sorted in the same order as the candidates.
    * @throws IOException
    */
-  public static void evaluate(List<? extends Rule> rules, Path inpath,
-      List<CDFitness> evaluations) throws IOException {
-    evaluate(rules, inpath, evaluations, new DatasetSplit(1));
+  public static void evaluate(List<? extends Rule> rules, int target,
+      Path inpath, List<CDFitness> evaluations) throws IOException {
+    evaluate(rules, target, inpath, evaluations, new DatasetSplit(1));
   }
 
   /**
    * Configure the job
    * 
-   * @param conf
-   * @param inpath input <code>Path</code>
+   * @param conf Job to configure
+   * @param rules classification rules to evaluate
+   * @param target label value to evaluate the rules for
+   * @param inpath input path (the dataset)
    * @param outpath output <code>Path</code>
+   * @param split DatasetSplit used to separate training and testing input
    */
   private static void configureJob(JobConf conf, List<? extends Rule> rules,
-      Path inpath, Path outpath, DatasetSplit split) {
+      int target, Path inpath, Path outpath, DatasetSplit split) {
     split.storeJobParameters(conf);
 
     DatasetTextInputFormat.setInputPaths(conf, inpath);
@@ -138,17 +153,22 @@
     conf.setInputFormat(DatasetTextInputFormat.class);
     conf.setOutputFormat(SequenceFileOutputFormat.class);
 
-    // store the stringified rules
+    // store the parameters
     conf.set(CDMapper.CLASSDISCOVERY_RULES, StringUtils.toString(rules));
+    conf.set(CDMapper.CLASSDISCOVERY_DATASET, StringUtils.toString(DataSet
+        .getDataSet()));
+    conf.setInt(CDMapper.CLASSDISCOVERY_TARGET_LABEL, target);
   }
 
   /**
    * Reads back the evaluations.
    * 
-   * @param fs
-   * @param conf
+   * @param fs File System
+   * @param conf Job configuration
    * @param outpath output <code>Path</code>
-   * @param evaluations List of evaluations
+   * @param evaluations <code>List&lt;Fitness&gt;</code> that contains the
+   *        evaluated fitness for each candidate from the input population,
+   *        sorted in the same order as the candidates.
    * @throws IOException
    */
   private static void importEvaluations(FileSystem fs, JobConf conf,

Modified: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapper.java?rev=686221&r1=686220&r2=686221&view=diff
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapper.java (original)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/hadoop/CDMapper.java Fri Aug 15 07:09:36 2008
@@ -30,18 +30,25 @@
 import org.apache.mahout.utils.StringUtils;
 import org.apache.mahout.ga.watchmaker.cd.CDFitness;
 import org.apache.mahout.ga.watchmaker.cd.DataLine;
+import org.apache.mahout.ga.watchmaker.cd.DataSet;
 import org.apache.mahout.ga.watchmaker.cd.Rule;
 
 /**
- * Hadoop Mapper. Evaluate all the rules with the input data line.  
+ * Hadoop Mapper. Evaluate all the rules with the input data line.
  */
 public class CDMapper extends MapReduceBase implements
     Mapper<LongWritable, Text, LongWritable, CDFitness> {
 
   public static final String CLASSDISCOVERY_RULES = "mahout.ga.classdiscovery.rules";
 
+  public static final String CLASSDISCOVERY_DATASET = "mahout.ga.classdiscovery.dataset";
+
+  public static final String CLASSDISCOVERY_TARGET_LABEL = "mahout.ga.classdiscovery.target";
+
   private List<Rule> rules;
 
+  private int target;
+
   @Override
   public void configure(JobConf job) {
     String rstr = job.get(CLASSDISCOVERY_RULES);
@@ -49,38 +56,66 @@
       throw new RuntimeException("Job Parameter (" + CLASSDISCOVERY_RULES
           + ") not found!");
 
-    configure((List<Rule>) StringUtils.fromString(rstr));
+    String datastr = job.get(CLASSDISCOVERY_DATASET);
+    if (datastr == null)
+      throw new RuntimeException("Job Parameter (" + CLASSDISCOVERY_DATASET
+          + ") not found!");
+
+    int target = job.getInt(CLASSDISCOVERY_TARGET_LABEL, -1);
+    if (target == -1)
+      throw new RuntimeException("Job Parameter ("
+          + CLASSDISCOVERY_TARGET_LABEL + ") not found!");
+
+    initializeDataSet((DataSet) StringUtils.fromString(datastr));
+    configure((List<Rule>) StringUtils.fromString(rstr), target);
 
     super.configure(job);
   }
 
-  void configure(List<Rule> rules) {
+  void initializeDataSet(DataSet dataset) {
+    assert dataset != null : "bad 'dataset' configuration parameter";
+    DataSet.initialize(dataset);
+  }
+
+  void configure(List<Rule> rules, int target) {
+    assert rules != null && !rules.isEmpty() : "bad 'rules' configuration parameter";
+    assert target >= 0 : "bad 'target' configuration parameter";
+
     this.rules = rules;
+    this.target = target;
+
   }
 
   public void map(LongWritable key, Text value,
       OutputCollector<LongWritable, CDFitness> output, Reporter reporter)
       throws IOException {
     DataLine dl = new DataLine(value.toString());
-    
+
     map(key, dl, output);
   }
 
   void map(LongWritable key, DataLine dl,
       OutputCollector<LongWritable, CDFitness> output) throws IOException {
     for (int index = 0; index < rules.size(); index++) {
-      CDFitness eval = evaluate(rules.get(index).classify(dl), dl.getLabel());
+      CDFitness eval = evaluate(target, rules.get(index).classify(dl), dl
+          .getLabel());
       output.collect(new LongWritable(index), eval);
     }
   }
 
-  public static CDFitness evaluate(int prediction, int label) {
-    // TODO for now we assume their are only two classes 0 and 1
-
-    int tp = (label == 1 && prediction == 1) ? 1 : 0;
-    int fp = (label == 0 && prediction == 1) ? 1 : 0;
-    int tn = (label == 0 && prediction == 0) ? 1 : 0;
-    int fn = (label == 1 && prediction == 0) ? 1 : 0;
+  /**
+   * Evaluate a given prediction.
+   * 
+   * @param target expected label
+   * @param prediction
+   * @param label actual label
+   * @return
+   */
+  public static CDFitness evaluate(int target, int prediction, int label) {
+    int tp = (label == target && prediction == 1) ? 1 : 0;
+    int fp = (label != target && prediction == 1) ? 1 : 0;
+    int tn = (label != target && prediction == 0) ? 1 : 0;
+    int fn = (label == target && prediction == 0) ? 1 : 0;
 
     return new CDFitness(tp, fp, tn, fn);
   }

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosTool.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosTool.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosTool.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosTool.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,219 @@
+package org.apache.mahout.ga.watchmaker.cd.tool;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Scanner;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Sorter;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.mahout.ga.watchmaker.OutputUtils;
+import org.apache.mahout.ga.watchmaker.cd.FileInfoParser;
+import org.apache.mahout.utils.StringUtils;
+
+/**
+ * Gathers additional information about a given dataset. Takes a descriptor
+ * about the attributes, and generates a description for each one.
+ * 
+ */
+public class CDInfosTool {
+
+  /**
+   * Uses Mahout to gather the information about a dataset.
+   * 
+   * @param descriptors about the available attributes
+   * @param inpath input path (the dataset)
+   * @param descriptions <code>List&lt;String&gt;</code> that contains the
+   *        generated descriptions for each non ignored attribute
+   * @throws IOException
+   */
+  public static void gatherInfos(Descriptors descriptors, Path inpath,
+      List<String> descriptions) throws IOException {
+    JobConf conf = new JobConf(CDInfosTool.class);
+    FileSystem fs = FileSystem.get(conf);
+
+    // check the input
+    if (!fs.exists(inpath) || !fs.getFileStatus(inpath).isDir())
+      throw new RuntimeException("Input path not found or is not a directory");
+
+    Path outpath = OutputUtils.prepareOutput(fs);
+
+    configureJob(conf, descriptors, inpath, outpath);
+    JobClient.runJob(conf);
+
+    importDescriptions(fs, conf, outpath, descriptions);
+  }
+
+  /**
+   * Configure the job
+   * 
+   * @param conf
+   * @param descriptors attributes's descriptors
+   * @param inpath input <code>Path</code>
+   * @param outpath output <code>Path</code>
+   */
+  private static void configureJob(JobConf conf, Descriptors descriptors,
+      Path inpath, Path outpath) {
+    TextInputFormat.setInputPaths(conf, inpath);
+    SequenceFileOutputFormat.setOutputPath(conf, outpath);
+
+    conf.setOutputKeyClass(LongWritable.class);
+    conf.setOutputValueClass(Text.class);
+
+    conf.setMapperClass(ToolMapper.class);
+    conf.setCombinerClass(ToolCombiner.class);
+    conf.setReducerClass(ToolReducer.class);
+
+    conf.setInputFormat(TextInputFormat.class);
+    conf.setOutputFormat(SequenceFileOutputFormat.class);
+
+    // store the stringified descriptors
+    conf.set(ToolMapper.ATTRIBUTES, StringUtils.toString(descriptors.getChars()));
+  }
+
+  /**
+   * Reads back the descriptions.
+   * 
+   * @param fs file system
+   * @param conf job configuration
+   * @param outpath output <code>Path</code>
+   * @param descriptions List of attribute's descriptions
+   * @throws IOException
+   */
+  private static void importDescriptions(FileSystem fs, JobConf conf,
+      Path outpath, List<String> descriptions) throws IOException {
+    Sorter sorter = new Sorter(fs, LongWritable.class, Text.class, conf);
+
+    // merge and sort the outputs
+    Path[] outfiles = OutputUtils.listOutputFiles(fs, outpath);
+    Path output = new Path(outpath, "output.sorted");
+    sorter.merge(outfiles, output);
+
+    // import the descriptions
+    LongWritable key = new LongWritable();
+    Text value = new Text();
+    Reader reader = new Reader(fs, output, conf);
+
+    while (reader.next(key, value)) {
+      descriptions.add(value.toString());
+    }
+
+    reader.close();
+  }
+
+  /**
+   * Load the dataset's attributes descriptors from an .info file
+   * 
+   * @param inpath dataset path
+   * @return
+   * @throws IOException
+   */
+  private static Descriptors loadDescriptors(FileSystem fs, Path inpath)
+      throws IOException {
+    // TODO should become part of FileInfoParser
+
+    Path infpath = FileInfoParser.getInfoFile(fs, inpath);
+
+    FSDataInputStream input = fs.open(infpath);
+    Scanner reader = new Scanner(input);
+
+    List<Character> descriptors = new ArrayList<Character>();
+
+    while (reader.hasNextLine()) {
+      String c = reader.nextLine();
+        descriptors.add(c.toUpperCase().charAt(0));
+    }
+
+    if (descriptors.isEmpty()) {
+      throw new RuntimeException("Infos file is empty");
+    }
+
+    char[] desc = new char[descriptors.size()];
+    for (int index = 0; index < descriptors.size(); index++) {
+      desc[index] = descriptors.get(index);
+    }
+
+    return new Descriptors(desc);
+  }
+
+  private static void storeDescriptions(FileSystem fs, Path inpath,
+      Descriptors descriptors, List<String> descriptions) throws IOException {
+    // TODO should become part of FileInfoParser
+
+    Path infpath = FileInfoParser.getInfoFile(fs, inpath);
+
+    FSDataOutputStream out = fs.create(infpath);
+    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+
+    int aindex = 0;
+    for (int index = 0; index < descriptors.size(); index++) {
+      if (descriptors.isLabel(index)) {
+        writer.write(FileInfoParser.LABEL_TOKEN + ", ");
+        writer.write(descriptions.get(aindex++));
+      } else if (descriptors.isNumerical(index)) {
+        writer.write(FileInfoParser.NUMERICAL_TOKEN + ", ");
+        writer.write(descriptions.get(aindex++));
+      } else if (descriptors.isNominal(index)) {
+        writer.write(FileInfoParser.NOMINAL_TOKEN + ", ");
+        writer.write(descriptions.get(aindex++));
+      } else {
+        writer.write(FileInfoParser.IGNORED_TOKEN);
+      }
+      
+      writer.newLine();
+    }
+
+    writer.close();
+  }
+
+  public static void main(String[] args) throws IOException {
+    // command-line parameters
+    if (args.length == 0) {
+      System.out.println("Usage: CDInfosTool dataset_path");
+      System.exit(-1);
+    }
+
+    FileSystem fs = FileSystem.get(new Configuration());
+    Path inpath = new Path(args[0]);
+
+    System.out.println("Loading Descriptors...");
+    Descriptors descriptors = loadDescriptors(fs, inpath);
+
+    System.out.println("Gathering informations...");
+    List<String> descriptions = new ArrayList<String>();
+    gatherInfos(descriptors, inpath, descriptions);
+
+    System.out.println("Storing Descriptions...");
+    storeDescriptions(fs, inpath, descriptors, descriptions);
+  }
+}

Propchange: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/CDInfosTool.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtils.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtils.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtils.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtils.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.ga.watchmaker.cd.tool;
+
+import java.util.Collection;
+import java.util.StringTokenizer;
+
+/**
+ * Utility functions to handle Attribute's description strings.
+ */
+public class DescriptionUtils {
+
+  public static class Range {
+    public final float min;
+    public final float max;
+    
+    public Range(float min, float max) {
+      this.max = max;
+      this.min = min;
+    }
+  }
+  
+  /**
+   * Create a numerical attribute description.
+   * 
+   * @param min
+   * @param max
+   * @return
+   */
+  public static String createNumericalDescription(float min, float max) {
+    return min + "," + max;
+  }
+
+  /**
+   * Create a nominal description from the possible values.
+   * 
+   * @param values
+   * @return
+   */
+  public static String createNominalDescription(Collection<String> values) {
+    StringBuffer buffer = new StringBuffer();
+    int ind = 0;
+
+    for (String value : values) {
+      buffer.append(value);
+      if (++ind < values.size())
+        buffer.append(",");
+    }
+
+    return buffer.toString();
+  }
+
+  public static Range extractNumericalRange(String description) {
+    StringTokenizer tokenizer = new StringTokenizer(description, ",");
+    float min = Float.valueOf(tokenizer.nextToken());
+    float max = Float.valueOf(tokenizer.nextToken());
+    
+    return new Range(min, max);
+  }
+  /**
+   * Extract all available values from the description.
+   * 
+   * @param description
+   * @param target the extracted values will be added to this collection. It
+   *        will not be cleared.
+   */
+  public static void extractNominalValues(String description,
+      Collection<String> target) {
+    StringTokenizer tokenizer = new StringTokenizer(description, ",");
+    while (tokenizer.hasMoreTokens()) {
+      target.add(tokenizer.nextToken());
+    }
+  }
+
+}

Propchange: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/DescriptionUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/Descriptors.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/Descriptors.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/Descriptors.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/Descriptors.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.ga.watchmaker.cd.tool;
+
+/**
+ * Used as a configuration Parameters for the mapper-combiner-reducer<br>
+ * CDTOOL.ATTRIBUTES : char[] description of the attributes.<br>
+ * for each attribute, takes one of this values:<br>
+ * <ul>
+ * <li>N : numerical attribute</li>
+ * <li>C : categorical (nominal) attribute</li>
+ * <li>L : label (nominal) attribute</li>
+ * <li>I : ignored attribute</li>
+ * </ul>
+ * 
+ */
+public class Descriptors {
+
+  private char[] descriptors;
+
+  public Descriptors(char[] descriptors) {
+    assert descriptors != null && descriptors.length > 0;
+
+    this.descriptors = descriptors;
+
+    // check that all the descriptors are correct ('I', 'N' 'L' or 'C')
+    for (int index = 0; index < descriptors.length; index++) {
+      if (!isIgnored(index) && !isNumerical(index) && !isNominal(index))
+        throw new RuntimeException("Bad descriptor value : "
+            + descriptors[index]);
+    }
+  }
+
+  public boolean isIgnored(int index) {
+    return descriptors[index] == 'i' || descriptors[index] == 'I';
+  }
+
+  public boolean isNumerical(int index) {
+    return descriptors[index] == 'n' || descriptors[index] == 'N';
+  }
+
+  public boolean isNominal(int index) {
+    return descriptors[index] == 'c' || descriptors[index] == 'C' || isLabel(index);
+  }
+
+  public boolean isLabel(int index) {
+    return descriptors[index] == 'l' || descriptors[index] == 'L';
+  }
+  
+  public int size() {
+    return descriptors.length;
+  }
+  
+  public char[] getChars() {
+    return descriptors;
+  }
+}

Propchange: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/Descriptors.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.ga.watchmaker.cd.tool;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.utils.StringUtils;
+
+/**
+ * Combines attribute values into a String.<br>
+ * <ul>
+ * <li>For Numerical attributes, the string contains the min and max values
+ * found.</li>
+ * <li>For Categorical attributes, the string contains the distinct values
+ * found.</li>
+ * </ul>
+ * 
+ * @see Descriptors, for more informations about the job parameter
+ */
+public class ToolCombiner extends MapReduceBase implements
+    Reducer<LongWritable, Text, LongWritable, Text> {
+
+  private Set<String> distinct = new HashSet<String>();
+
+  private Descriptors descriptors;
+
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+
+    String descriptors = job.get(ToolMapper.ATTRIBUTES);
+
+    if (descriptors != null)
+      configure((char[]) StringUtils.fromString(descriptors));
+  }
+
+  void configure(char[] descriptors) {
+    if (descriptors == null || descriptors.length == 0)
+      throw new RuntimeException("Descriptors's array not found or is empty");
+
+    this.descriptors = new Descriptors(descriptors);
+  }
+
+  public void reduce(LongWritable key, Iterator<Text> values,
+      OutputCollector<LongWritable, Text> output, Reporter reporter)
+      throws IOException {
+    output.collect(key, new Text(createDescription((int) key.get(), values)));
+  }
+
+  /**
+   * Generate a String description for a given attribute from its available
+   * values.
+   * 
+   * @param index attribute index
+   * @param values available values
+   * @return
+   * @throws RuntimeException if the attribute should be ignored.
+   */
+  String createDescription(int index, Iterator<Text> values) {
+    if (descriptors.isNominal(index))
+      return nominalDescription(values);
+    else if (descriptors.isNumerical(index))
+      return numericalDescription(values);
+    else
+      throw new RuntimeException(
+          "An ignored attribute should never reach the Combiner");
+  }
+
+  String nominalDescription(Iterator<Text> values) {
+    // distinct values
+    distinct.clear();
+    while (values.hasNext()) {
+      distinct.add(values.next().toString());
+    }
+
+    return DescriptionUtils.createNominalDescription(distinct);
+  }
+
+  String numericalDescription(Iterator<Text> values) {
+    float min = Float.MAX_VALUE;
+    float max = Float.MIN_VALUE;
+
+    while (values.hasNext()) {
+      float value = Float.valueOf(values.next().toString());
+      if (value < min)
+        min = value;
+      else if (value > max)
+        max = value;
+    }
+
+    return DescriptionUtils.createNumericalDescription(min, max);
+  }
+}

Propchange: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolCombiner.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapper.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapper.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapper.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.ga.watchmaker.cd.tool;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.utils.StringUtils;
+
+/**
+ * Extract the attribute values from a dataline. Skip ignored attributes<br>
+ * Input:<br>
+ * <ul>
+ * <li> LongWritable : data row index </li>
+ * <li> Text : dataline </li>
+ * </ul>
+ * Output:<br>
+ * <ul>
+ * <li> LongWritable : attribute index.<br>
+ * ignored attributes aren't taken into account when calculating this index.</li>
+ * <li> Text : attribute value </li>
+ * </ul>
+ * 
+ * @see Descriptors, for more informations about the job parameter
+ */
+public class ToolMapper extends MapReduceBase implements
+    Mapper<LongWritable, Text, LongWritable, Text> {
+
+  public static final String ATTRIBUTES = "cdtool.attributes";
+
+  private List<String> attributes = new ArrayList<String>();
+
+  private Descriptors descriptors;
+  
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+
+    String descrs = job.get(ATTRIBUTES);
+
+    if (descrs != null)
+      configure((char[]) StringUtils.fromString(descrs));
+  }
+
+  void configure(char[] descriptors) {
+    if (descriptors == null || descriptors.length == 0)
+      throw new RuntimeException("Descriptors's array not found or is empty");
+
+    this.descriptors = new Descriptors(descriptors);
+  }
+
+  public void map(LongWritable key, Text value,
+      OutputCollector<LongWritable, Text> output, Reporter reporter)
+      throws IOException {
+    extractAttributes(value, attributes);
+    if (attributes.size() != descriptors.size())
+      throw new RuntimeException(
+          "Attributes number should be equal to the descriptors's array length");
+
+    // output non ignored attributes
+    for (int index = 0; index < attributes.size(); index++) {
+      if (descriptors.isIgnored(index))
+        continue;
+
+      output.collect(new LongWritable(index), new Text(attributes.get(index)));
+    }
+  }
+
+  /**
+   * Extract attribute values from the input Text. The attributes are separated
+   * by a colon ','. Skips ignored attributes.
+   * 
+   * @param value
+   * @param attributes
+   */
+  void extractAttributes(Text value, List<String> attributes) {
+    StringTokenizer tokenizer = new StringTokenizer(value.toString(), ",");
+
+    attributes.clear();
+    while (tokenizer.hasMoreTokens()) {
+      attributes.add(tokenizer.nextToken().trim());
+    }
+  }
+}

Propchange: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolMapper.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java?rev=686221&view=auto
==============================================================================
--- lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java (added)
+++ lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java Fri Aug 15 07:09:36 2008
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.mahout.ga.watchmaker.cd.tool;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.MapReduceBase;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.mahout.ga.watchmaker.cd.tool.DescriptionUtils.Range;
+import org.apache.mahout.utils.StringUtils;
+
+/**
+ * Combines attribute description strings into a String.<br>
+ * <ul>
+ * <li>For Numerical attributes, the string contains the min and max values
+ * found.</li>
+ * <li>For Categorical attributes, the string contains the distinct values
+ * found.</li>
+ * </ul>
+ * 
+ * @see Descriptors, for more informations about the job parameter
+ */
+public class ToolReducer extends MapReduceBase implements
+    Reducer<LongWritable, Text, LongWritable, Text> {
+
+  private Descriptors descriptors;
+
+  private Set<String> distinct = new HashSet<String>();
+
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+
+    String descriptors = job.get(ToolMapper.ATTRIBUTES);
+
+    if (descriptors != null)
+      configure((char[]) StringUtils.fromString(descriptors));
+  }
+
+  void configure(char[] descriptors) {
+    if (descriptors == null || descriptors.length == 0)
+      throw new RuntimeException("Descriptors's array not found or is empty");
+
+    this.descriptors = new Descriptors(descriptors);
+  }
+
+  public void reduce(LongWritable key, Iterator<Text> values,
+      OutputCollector<LongWritable, Text> output, Reporter reporter)
+      throws IOException {
+    output.collect(key, new Text(combineDescriptions((int) key.get(), values)));
+  }
+
+  /**
+   * Combines a given attribute descriptions into a single descriptor.
+   * 
+   * @param index attribute index
+   * @param values available descriptions
+   * @return
+   * @throws RuntimeException if the attribute should be ignored.
+   */
+  String combineDescriptions(int index, Iterator<Text> values) {
+    if (descriptors.isNumerical(index))
+      return numericDescription(values);
+    else if (descriptors.isNominal(index))
+      return nominalDescription(values);
+    else
+      throw new RuntimeException("");
+  }
+
+  String numericDescription(Iterator<Text> values) {
+    float min = Float.MAX_VALUE;
+    float max = Float.MIN_VALUE;
+
+    while (values.hasNext()) {
+      Range range = DescriptionUtils.extractNumericalRange(values.next().toString());
+      min = Math.min(min, range.min);
+      max = Math.max(max, range.max);
+    }
+
+    return DescriptionUtils.createNumericalDescription(min, max);
+  }
+
+  String nominalDescription(Iterator<Text> values) {
+    distinct.clear();
+
+    // extract all distinct values
+    while (values.hasNext()) {
+      DescriptionUtils.extractNominalValues(values.next().toString(), distinct);
+    }
+
+    // create a new description
+    return DescriptionUtils.createNominalDescription(distinct);
+  }
+}

Propchange: lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/ga/watchmaker/cd/tool/ToolReducer.java
------------------------------------------------------------------------------
    svn:eol-style = native