You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mahout.apache.org by je...@apache.org on 2009/03/16 01:17:08 UTC

svn commit: r754797 [1/2] - in /lucene/mahout/trunk: ./ core/ core/src/main/java/org/apache/mahout/clustering/dirichlet/ core/src/main/java/org/apache/mahout/clustering/dirichlet/models/ core/src/test/java/org/apache/mahout/clustering/dirichlet/ exampl...

Author: jeastman
Date: Mon Mar 16 00:17:07 2009
New Revision: 754797

URL: http://svn.apache.org/viewvc?rev=754797&view=rev
Log:
- Committing Dirichlet Process Clustering (MAHOUT-30). Some minor refactoring from the latest patch (MAHOUT-30f) to move models and distributions from test tree into core so that examples can see them.
All tests run and the pom packages.

Added:
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCluster.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCombiner.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletJob.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletMapper.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletReducer.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletState.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonDirichletStateAdapter.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelDistributionAdapter.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonVectorAdapter.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/UncommonDistributions.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalDistribution.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalModel.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/Model.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/ModelDistribution.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModel.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModelDistribution.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/SampledNormalDistribution.java
    lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/SampledNormalModel.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/JsonModelHolderAdapter.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/ModelHolder.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestDirichletClustering.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestDistributions.java
    lucene/mahout/trunk/core/src/test/java/org/apache/mahout/clustering/dirichlet/TestMapReduce.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/Display2dASNDirichlet.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNDirichlet.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayASNOutputState.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayDirichlet.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayNDirichlet.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplayOutputState.java
    lucene/mahout/trunk/examples/src/main/java/org/apache/mahout/clustering/dirichlet/DisplaySNDirichlet.java
Modified:
    lucene/mahout/trunk/LICENSE.txt
    lucene/mahout/trunk/NOTICE.txt
    lucene/mahout/trunk/core/pom.xml

Modified: lucene/mahout/trunk/LICENSE.txt
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/LICENSE.txt?rev=754797&r1=754796&r2=754797&view=diff
==============================================================================
--- lucene/mahout/trunk/LICENSE.txt (original)
+++ lucene/mahout/trunk/LICENSE.txt Mon Mar 16 00:17:07 2009
@@ -479,4 +479,41 @@
 
 This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation.
 
+==========================================================
+The following license applies to the gamma() method in the UncommonDistributions class
+==========================================================
+
+Copyright (c) 2005, Regents of the University of California
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions
+are met:
+
+* Redistributions of source code must retain the above copyright
+  notice, this list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright
+  notice, this list of conditions and the following disclaimer in
+  the documentation and/or other materials provided with the
+  distribution.  
+
+* Neither the name of the University of California, Berkeley nor
+  the names of its contributors may be used to endorse or promote
+  products derived from this software without specific prior 
+  written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS
+FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE
+COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT,
+INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR
+SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION)
+HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
+OF THE POSSIBILITY OF SUCH DAMAGE.
+
 

Modified: lucene/mahout/trunk/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/NOTICE.txt?rev=754797&r1=754796&r2=754797&view=diff
==============================================================================
--- lucene/mahout/trunk/NOTICE.txt (original)
+++ lucene/mahout/trunk/NOTICE.txt Mon Mar 16 00:17:07 2009
@@ -32,3 +32,6 @@
 
 This product uses the Breast Cancer Wisconsin (Diagnostic) Dataset from the
 UCI Machine Learning Repository [http://www.ics.uci.edu/~mlearn/MLRepository.html]. 
+
+This product contains parts of the Bayesian Logic (BLOG) Inference Engine.
+Copyright (c) 2005, Regents of the University of California

Modified: lucene/mahout/trunk/core/pom.xml
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/pom.xml?rev=754797&r1=754796&r2=754797&view=diff
==============================================================================
--- lucene/mahout/trunk/core/pom.xml (original)
+++ lucene/mahout/trunk/core/pom.xml Mon Mar 16 00:17:07 2009
@@ -563,7 +563,15 @@
       <version>3.8.2</version>
       <scope>test</scope>
     </dependency>
-
+    
+    <!--  Gson: Java to Json conversion -->
+    <dependency>
+      <groupId>com.google.code.gson</groupId>
+      <artifactId>gson</artifactId>
+      <version>1.3</version>
+      <scope>compile</scope>
+    </dependency>
+    
   </dependencies>
   <repositories>
     <repository>
@@ -575,7 +583,7 @@
       <releases>
         <enabled>false</enabled>
       </releases>
-    </repository>
+    </repository>    
   </repositories>
 
 

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCluster.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCluster.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCluster.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCluster.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,69 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.lang.reflect.Type;
+
+import org.apache.mahout.clustering.dirichlet.models.Model;
+import org.apache.mahout.matrix.Vector;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.reflect.TypeToken;
+
+public class DirichletCluster<Observation> {
+
+  public Model<Observation> model; // the model for this iteration
+
+  public double totalCount; // total count of observations for the model
+
+  public DirichletCluster(Model<Observation> model, double totalCount) {
+    super();
+    this.model = model;
+    this.totalCount = totalCount;
+  }
+
+  public DirichletCluster() {
+    super();
+  }
+
+  public void setModel(Model<Observation> model) {
+    this.model = model;
+    this.totalCount += model.count();
+  }
+
+  static Type typeOfModel = new TypeToken<DirichletCluster<Vector>>() {
+  }.getType();
+
+  public String asFormatString() {
+    GsonBuilder builder = new GsonBuilder();
+    builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
+    builder.registerTypeAdapter(Model.class, new JsonModelAdapter());
+    Gson gson = builder.create();
+    return gson.toJson(this, typeOfModel);
+  }
+
+  @SuppressWarnings("unchecked")
+  public static DirichletCluster fromFormatString(String formatString) {
+    GsonBuilder builder = new GsonBuilder();
+    builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
+    builder.registerTypeAdapter(Model.class, new JsonModelAdapter());
+    Gson gson = builder.create();
+    return gson.fromJson(formatString, typeOfModel);
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterer.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterer.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletClusterer.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,186 @@
+package org.apache.mahout.clustering.dirichlet;
+
+/**
+ * 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 org.apache.mahout.clustering.dirichlet.models.Model;
+import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
+import org.apache.mahout.matrix.DenseVector;
+import org.apache.mahout.matrix.TimesFunction;
+import org.apache.mahout.matrix.Vector;
+
+/**
+ * Performs Bayesian mixture modeling.
+ * <p/>
+ * The idea is that we use a probabilistic mixture of a number of models that we use to explain some observed data.
+ * The idea here is that each observed data point is assumed to have come from one of the models in the mixture, but
+ * we don't know which.  The way we deal with that is to use a so-called latent parameter which specifies which model
+ * each data point came from.
+ * <p/>
+ * In addition, since this is a Bayesian clustering algorithm, we don't want to actually commit to any single explanation,
+ * but rather to sample from the distribution of models and latent assignments of data points to models given the
+ * observed data and the prior distributions of model parameters.
+ * <p/>
+ * This sampling process is initialized by taking models at random from the prior distribution for models.
+ * <p/>
+ * Then, we iteratively assign points to the different models using the mixture probabilities and the degree
+ * of fit between the point and each model expressed as a probability that the point was generated by that model.
+ * <p/>
+ * After points are assigned, new parameters for each model are sampled from the posterior distribution for the model
+ * parameters considering all of the observed data points that were assigned to the model.  Models without any
+ * data points are also sampled, but since they have no points assigned, the new samples are effectively taken from
+ * the prior distribution for model parameters.
+ * <p/>
+ * The result is a number of samples that represent mixing probabilities, models and assignment of points to models.
+ * If the total number of possible models is substantially larger than the number that ever have points assigned to
+ * them, then this algorithm provides a (nearly) non-parametric clustering algorithm.
+ * <p/>
+ * These samples can give us interesting information that is lacking from a normal clustering that consists of a
+ * single assignment of points to clusters.  Firstly, by examining the number of models in each sample that actually
+ * has any points assigned to it, we can get information about how many models (clusters) that the data support.
+ * <p/>
+ * Morevoer, by examining how often two points are assigned to the same model, we can get an approximate measure
+ * of how likely these points are to be explained by the same model.  Such soft membership information is difficult
+ * to come by with conventional clustering methods.
+ * <p/>
+ * Finally, we can get an idea of the stability of how the data can be described.  Typically, aspects of the data
+ * with lots of data available wind up with stable descriptions while at the edges, there are aspects that are
+ * phenomena that we can't really commit to a solid description, but it is still clear that the well supported
+ * explanations are insufficient to explain these additional aspects.
+ * <p/>
+ * One thing that can be difficult about these samples is that we can't always assign a correlation between the models
+ * in the different samples.  Probably the best way to do this is to look for overlap in the assignments of data
+ * observations to the different models.
+ * <p/>
+ * <pre>
+ *    \theta_i ~ prior()
+ *    \lambda_i ~ Dirichlet(\alpha_0)
+ *    z_j ~ Multinomial( \lambda )
+ *    x_j ~ model(\theta_i)
+ * </pre>
+ */
+public class DirichletClusterer<Observation> {
+
+  // observed data
+  private List<Observation> sampleData;
+
+  // the ModelDistribution for the computation
+  private ModelDistribution<Observation> modelFactory;
+
+  // the state of the clustering process
+  private DirichletState<Observation> state;
+
+  private int thin;
+
+  private int burnin;
+
+  private int numClusters;
+
+  public List<Model<Observation>[]> clusterSamples = new ArrayList<Model<Observation>[]>();
+
+  /**
+   * Create a new instance on the sample data with the given additional parameters
+   *
+   * @param sampleData    the observed data to be clustered
+   * @param modelFactory  the ModelDistribution to use
+   * @param alpha_0       the double value for the beta distributions
+   * @param numClusters   the int number of clusters
+   * @param thin          the int thinning interval, used to report every n iterations
+   * @param burnin        the int burnin interval, used to suppress early iterations
+   */
+  public DirichletClusterer(List<Observation> sampleData,
+      ModelDistribution<Observation> modelFactory, double alpha_0,
+      int numClusters, int thin, int burnin) {
+    this.sampleData = sampleData;
+    this.modelFactory = modelFactory;
+    this.thin = thin;
+    this.burnin = burnin;
+    this.numClusters = numClusters;
+    state = new DirichletState<Observation>(modelFactory, numClusters, alpha_0,
+        thin, burnin);
+  }
+
+  /**
+   * Iterate over the sample data, obtaining cluster samples periodically
+   * and returning them.
+   * 
+   * @param numIterations the int number of iterations to perform
+   * @return a List<List<Model<Observation>>> of the observed models
+   */
+  public List<Model<Observation>[]> cluster(int numIterations) {
+    for (int iteration = 0; iteration < numIterations; iteration++)
+      iterate(iteration, state);
+    return clusterSamples;
+  }
+
+  /**
+   * Perform one iteration of the clustering process, iterating over the samples to build
+   * a new array of models, then updating the state for the next iteration
+   * 
+   * @param state the DirichletState<Observation> of this iteration
+   */
+  private void iterate(int iteration, DirichletState<Observation> state) {
+
+    // create new posterior models
+    Model<Observation>[] newModels = modelFactory.sampleFromPosterior(state
+        .getModels());
+
+    // iterate over the samples, assigning each to a model
+    for (Observation x : sampleData) {
+      // compute normalized vector of probabilities that x is described by each model
+      Vector pi = normalizedProbabilities(state, x);
+      // then pick one cluster by sampling a Multinomial distribution based upon them
+      // see: http://en.wikipedia.org/wiki/Multinomial_distribution
+      int k = UncommonDistributions.rMultinom(pi);
+      // ask the selected model to observe the datum
+      newModels[k].observe(x);
+    }
+
+    // periodically add models to the cluster samples after the burn-in period
+    if ((iteration >= burnin) && (iteration % thin == 0))
+      clusterSamples.add(newModels);
+    // update the state from the new models
+    state.update(newModels);
+  }
+
+  /**
+   * Compute a normalized vector of probabilities that x is described
+   * by each model using the mixture and the model pdfs
+   * 
+   * @param state the DirichletState<Observation> of this iteration
+   * @param x an Observation
+   * @return the Vector of probabilities
+   */
+  private Vector normalizedProbabilities(DirichletState<Observation> state,
+      Observation x) {
+    Vector pi = new DenseVector(numClusters);
+    double max = 0;
+    for (int k = 0; k < numClusters; k++) {
+      double p = state.adjustedProbability(x, k);
+      pi.set(k, p);
+      if (max < p)
+        max = p;
+    }
+    // normalize the probabilities by largest observed value
+    pi.assign(new TimesFunction(), 1.0 / max);
+    return pi;
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCombiner.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCombiner.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCombiner.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletCombiner.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,42 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+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;
+
+public class DirichletCombiner extends MapReduceBase implements
+    Reducer<Text, Text, Text, Text> {
+
+  public void reduce(Text key, Iterator<Text> values,
+      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+  }
+
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletDriver.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,201 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SequenceFileOutputFormat;
+import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
+import org.apache.mahout.clustering.kmeans.KMeansDriver;
+import org.apache.mahout.matrix.Vector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DirichletDriver {
+
+  public static final String STATE_IN_KEY = "org.apache.mahout.clustering.dirichlet.stateIn";
+
+  public static final String MODEL_FACTORY_KEY = "org.apache.mahout.clustering.dirichlet.modelFactory";
+
+  public static final String NUM_CLUSTERS_KEY = "org.apache.mahout.clustering.dirichlet.numClusters";
+
+  public static final String ALPHA_0_KEY = "org.apache.mahout.clustering.dirichlet.alpha_0";
+
+  private static final Logger log = LoggerFactory
+      .getLogger(DirichletDriver.class);
+
+  private DirichletDriver() {
+  }
+
+  public static void main(String[] args) throws InstantiationException,
+      IllegalAccessException, ClassNotFoundException, IOException {
+    String input = args[0];
+    String output = args[1];
+    String modelFactory = args[2];
+    int numClusters = Integer.parseInt(args[3]);
+    int maxIterations = Integer.parseInt(args[4]);
+    double alpha_0 = Double.parseDouble(args[5]);
+    int numReducers = Integer.parseInt(args[6]);
+    runJob(input, output, modelFactory, numClusters, maxIterations, alpha_0,
+        numReducers);
+  }
+
+  /**
+   * Run the job using supplied arguments
+   *
+   * @param input           the directory pathname for input points
+   * @param output          the directory pathname for output points
+   * @param modelFactory    the String ModelDistribution class name to use
+   * @param numClusters       the number of models
+   * @param maxIterations   the maximum number of iterations
+   * @param alpha_0         the alpha_0 value for the DirichletDistribution
+   * @param numReducers     the number of Reducers desired
+   * @throws ClassNotFoundException 
+   * @throws IllegalAccessException 
+   * @throws InstantiationException 
+   * @throws IOException 
+   */
+  public static void runJob(String input, String output, String modelFactory,
+      int numClusters, int maxIterations, double alpha_0, int numReducers)
+      throws ClassNotFoundException, InstantiationException,
+      IllegalAccessException, IOException {
+
+    String stateIn = output + "/state-0";
+    writeInitialState(output, stateIn, modelFactory, numClusters, alpha_0);
+
+    for (int iteration = 0; iteration < maxIterations; iteration++) {
+      log.info("Iteration {}", iteration);
+      // point the output to a new directory per iteration
+      String stateOut = output + "/state-" + (iteration + 1);
+      runIteration(input, stateIn, stateOut, modelFactory, numClusters,
+          alpha_0, numReducers);
+      // now point the input to the old output directory
+      stateIn = stateOut;
+    }
+  }
+
+  private static void writeInitialState(String output, String stateIn,
+      String modelFactory, int numModels, double alpha_0)
+      throws ClassNotFoundException, InstantiationException,
+      IllegalAccessException, FileNotFoundException, IOException {
+    DirichletState<Vector> state = createState(modelFactory, numModels, alpha_0);
+    JobConf job = new JobConf(KMeansDriver.class);
+    FileSystem fs = FileSystem.get(job);
+    fs.delete(new Path(output), true);
+    for (int i = 0; i < numModels; i++) {
+      Path path = new Path(stateIn + "/part-" + i);
+      SequenceFile.Writer writer = new SequenceFile.Writer(fs, job, path,
+          Text.class, Text.class);
+      String stateString = state.clusters.get(i).asFormatString();
+      writer.append(new Text(Integer.toString(i)), new Text(stateString));
+      writer.close();
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public static DirichletState<Vector> createState(String modelFactory,
+      int numModels, double alpha_0) throws ClassNotFoundException,
+      InstantiationException, IllegalAccessException {
+    ClassLoader ccl = Thread.currentThread().getContextClassLoader();
+    Class<?> cl = ccl.loadClass(modelFactory);
+    ModelDistribution<Vector> factory = (ModelDistribution<Vector>) cl
+        .newInstance();
+    DirichletState<Vector> state = new DirichletState<Vector>(factory,
+        numModels, alpha_0, 1, 1);
+    return state;
+  }
+
+  /**
+   * Run the job using supplied arguments
+   *
+   * @param input         the directory pathname for input points
+   * @param stateIn       the directory pathname for input state
+   * @param stateOut      the directory pathname for output state
+   * @param modelFactory  the class name of the model factory class
+   * @param numClusters   the number of clusters
+   * @param alpha_0       alpha_0
+   * @param numReducers   the number of Reducers desired
+   */
+  public static void runIteration(String input, String stateIn,
+      String stateOut, String modelFactory, int numClusters, double alpha_0,
+      int numReducers) {
+    JobClient client = new JobClient();
+    JobConf conf = new JobConf(DirichletDriver.class);
+
+    conf.setOutputKeyClass(Text.class);
+    conf.setOutputValueClass(Text.class);
+
+    FileInputFormat.setInputPaths(conf, new Path(input));
+    Path outPath = new Path(stateOut);
+    FileOutputFormat.setOutputPath(conf, outPath);
+
+    conf.setMapperClass(DirichletMapper.class);
+    conf.setReducerClass(DirichletReducer.class);
+    conf.setNumReduceTasks(numReducers);
+    conf.setOutputFormat(SequenceFileOutputFormat.class);
+    conf.set(STATE_IN_KEY, stateIn);
+    conf.set(MODEL_FACTORY_KEY, modelFactory);
+    conf.set(NUM_CLUSTERS_KEY, Integer.toString(numClusters));
+    conf.set(ALPHA_0_KEY, Double.toString(alpha_0));
+
+    client.setConf(conf);
+    try {
+      JobClient.runJob(conf);
+    } catch (IOException e) {
+      log.warn(e.toString(), e);
+    }
+  }
+
+  /**
+   * Run the job using supplied arguments
+   *
+   * @param input            the directory pathname for input points
+   * @param stateIn          the directory pathname for input state
+   * @param output           the directory pathname for output points
+   */
+  public static void runClustering(String input, String stateIn, String output) {
+    JobClient client = new JobClient();
+    JobConf conf = new JobConf(DirichletDriver.class);
+
+    conf.setOutputKeyClass(Text.class);
+    conf.setOutputValueClass(Text.class);
+
+    FileInputFormat.setInputPaths(conf, new Path(input));
+    Path outPath = new Path(output);
+    FileOutputFormat.setOutputPath(conf, outPath);
+
+    conf.setMapperClass(DirichletMapper.class);
+    conf.setNumReduceTasks(0);
+
+    client.setConf(conf);
+    try {
+      JobClient.runJob(conf);
+    } catch (IOException e) {
+      log.warn(e.toString(), e);
+    }
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletJob.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletJob.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletJob.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletJob.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,71 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+
+public class DirichletJob {
+
+  private DirichletJob() {
+  }
+
+  public static void main(String[] args) throws IOException,
+      ClassNotFoundException, InstantiationException, IllegalAccessException {
+    String input = args[0];
+    String output = args[1];
+    String modelFactory = args[2];
+    int numModels = Integer.parseInt(args[3]);
+    int maxIterations = Integer.parseInt(args[4]);
+    double alpha_0 = Double.parseDouble(args[5]);
+    runJob(input, output, modelFactory, numModels, maxIterations, alpha_0);
+  }
+
+  /**
+   * Run the job using supplied arguments, deleting the output directory if it
+   * exists beforehand
+   * 
+   * @param input the directory pathname for input points
+   * @param output the directory pathname for output points
+   * @param modelFactory the ModelDistribution class name
+   * @param numModels the number of Models
+   * @param maxIterations the maximum number of iterations
+   * @param alpha_0 the alpha0 value for the DirichletDistribution
+   * @throws IllegalAccessException 
+   * @throws InstantiationException 
+   * @throws ClassNotFoundException 
+   */
+  public static void runJob(String input, String output, String modelFactory,
+      int numModels, int maxIterations, double alpha_0)
+      throws IOException, ClassNotFoundException, InstantiationException,
+      IllegalAccessException {
+    // delete the output directory
+    JobConf conf = new JobConf(DirichletJob.class);
+    Path outPath = new Path(output);
+    FileSystem fs = FileSystem.get(conf);
+    if (fs.exists(outPath)) {
+      fs.delete(outPath, true);
+    }
+    fs.mkdirs(outPath);
+    DirichletDriver.runJob(input, output, modelFactory, numModels, maxIterations,
+        alpha_0, 1);
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletMapper.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletMapper.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletMapper.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletMapper.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,122 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+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.matrix.DenseVector;
+import org.apache.mahout.matrix.TimesFunction;
+import org.apache.mahout.matrix.Vector;
+
+public class DirichletMapper extends MapReduceBase implements
+    Mapper<WritableComparable<?>, Text, Text, Text> {
+
+  DirichletState<Vector> state;
+
+  public void map(WritableComparable<?> key, Text values,
+      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+    Vector v = DenseVector.decodeFormat(values.toString());
+    // compute a normalized vector of probabilities that v is described by each model
+    Vector pi = normalizedProbabilities(state, v);
+    // then pick one model by sampling a Multinomial distribution based upon them
+    // see: http://en.wikipedia.org/wiki/Multinomial_distribution
+    int k = UncommonDistributions.rMultinom(pi);
+    output.collect(new Text("" + k), values);
+  }
+
+  public void configure(DirichletState<Vector> state) {
+    this.state = state;
+  }
+
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+    state = getDirichletState(job);
+  }
+
+  @SuppressWarnings("unchecked")
+  public static DirichletState<Vector> getDirichletState(JobConf job) {
+    String statePath = job.get(DirichletDriver.STATE_IN_KEY);
+    String modelFactory = job.get(DirichletDriver.MODEL_FACTORY_KEY);
+    String numClusters = job.get(DirichletDriver.NUM_CLUSTERS_KEY);
+    String alpha_0 = job.get(DirichletDriver.ALPHA_0_KEY);
+
+    try {
+      DirichletState<Vector> state = DirichletDriver.createState(modelFactory,
+          new Integer(numClusters), new Double(alpha_0));
+      FileSystem fs = FileSystem.get(job);
+      Path path = new Path(statePath);
+      FileStatus[] status = fs.listStatus(path);
+      for (FileStatus s : status) {
+        SequenceFile.Reader reader = new SequenceFile.Reader(fs, s.getPath(),
+            job);
+        try {
+          Text key = new Text();
+          Text value = new Text();
+          while (reader.next(key, value)) {
+            int index = new Integer(key.toString());
+            String formatString = value.toString();
+            DirichletCluster<Vector> cluster = DirichletCluster
+                .fromFormatString(formatString);
+            state.clusters.set(index, cluster);
+          }
+        } finally {
+          reader.close();
+        }
+      }
+      // TODO: with more than one mapper, they will all have different mixtures. Will this matter?
+      state.mixture = UncommonDistributions.rDirichlet(state.totalCounts());
+      return state;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Compute a normalized vector of probabilities that v is described
+   * by each model using the mixture and the model pdfs
+   * 
+   * @param state the DirichletState<Vector> of this iteration
+   * @param v an Vector
+   * @return the Vector of probabilities
+   */
+  private Vector normalizedProbabilities(DirichletState<Vector> state, Vector v) {
+    Vector pi = new DenseVector(state.numClusters);
+    double max = 0;
+    for (int k = 0; k < state.numClusters; k++) {
+      double p = state.adjustedProbability(v, k);
+      pi.set(k, p);
+      if (max < p)
+        max = p;
+    }
+    // normalize the probabilities by largest observed value
+    pi.assign(new TimesFunction(), 1.0 / max);
+    return pi;
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletReducer.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletReducer.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletReducer.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletReducer.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,66 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+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.clustering.dirichlet.models.Model;
+import org.apache.mahout.matrix.DenseVector;
+import org.apache.mahout.matrix.Vector;
+
+public class DirichletReducer extends MapReduceBase implements
+    Reducer<Text, Text, Text, Text> {
+
+  DirichletState<Vector> state;
+
+  public Model<Vector>[] newModels;
+
+  public void reduce(Text key, Iterator<Text> values,
+      OutputCollector<Text, Text> output, Reporter reporter) throws IOException {
+    Integer k = new Integer(key.toString());
+    Model<Vector> model = newModels[k];
+    while (values.hasNext()) {
+      Vector v = DenseVector.decodeFormat(values.next().toString());
+      model.observe(v);
+    }
+    model.computeParameters();
+    DirichletCluster<Vector> cluster = state.clusters.get(k);
+    cluster.setModel(model);
+    output.collect(key, new Text(cluster.asFormatString()));
+  }
+
+  public void configure(DirichletState<Vector> state) {
+    this.state = state;
+    this.newModels = state.modelFactory.sampleFromPosterior(state.getModels());
+  }
+
+  @Override
+  public void configure(JobConf job) {
+    super.configure(job);
+    state = DirichletMapper.getDirichletState(job);
+    this.newModels = state.modelFactory.sampleFromPosterior(state.getModels());
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletState.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletState.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletState.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/DirichletState.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,103 @@
+package org.apache.mahout.clustering.dirichlet;
+
+/**
+ * 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 org.apache.mahout.clustering.dirichlet.models.Model;
+import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
+import org.apache.mahout.matrix.DenseVector;
+import org.apache.mahout.matrix.Vector;
+
+public class DirichletState<Observation> {
+
+  public int numClusters; // the number of clusters
+
+  public ModelDistribution<Observation> modelFactory; // the factory for models
+
+  public List<DirichletCluster<Observation>> clusters; // the clusters for this iteration
+
+  public Vector mixture; // the mixture vector
+
+  public double offset; // alpha_0 / numClusters
+
+  @SuppressWarnings("unchecked")
+  public DirichletState(ModelDistribution<Observation> modelFactory,
+      int numClusters, double alpha_0, int thin, int burnin) {
+    this.numClusters = numClusters;
+    this.modelFactory = modelFactory;
+    // initialize totalCounts
+    offset = alpha_0 / numClusters;
+    // sample initial prior models
+    clusters = new ArrayList<DirichletCluster<Observation>>();
+    for (Model m : modelFactory.sampleFromPrior(numClusters))
+      clusters.add(new DirichletCluster(m, offset));
+    // sample the mixture parameters from a Dirichlet distribution on the totalCounts 
+    mixture = UncommonDistributions.rDirichlet(totalCounts());
+  }
+
+  public DirichletState() {
+  }
+
+  public Vector totalCounts() {
+    Vector result = new DenseVector(numClusters);
+    for (int i = 0; i < numClusters; i++)
+      result.set(i, clusters.get(i).totalCount);
+    return result;
+  }
+
+  /**
+   * Update the receiver with the new models
+   * 
+   * @param newModels a Model<Observation>[] of new models
+   */
+  public void update(Model<Observation>[] newModels) {
+    // compute new model parameters based upon observations and update models
+    for (int i = 0; i < newModels.length; i++) {
+      newModels[i].computeParameters();
+      clusters.get(i).setModel(newModels[i]);
+    }
+    // update the mixture
+    mixture = UncommonDistributions.rDirichlet(totalCounts());
+  }
+
+  /**
+   * return the adjusted probability that x is described by the kth model
+   * @param x an Observation
+   * @param k an int index of a model
+   * @return the double probability
+   */
+  public double adjustedProbability(Observation x, int k) {
+    double pdf = clusters.get(k).model.pdf(x);
+    double mix = mixture.get(k);
+    double result = mix * pdf;
+    if (result < 0 || result > 1)
+      System.out.print("");
+    return result;
+  }
+
+  @SuppressWarnings("unchecked")
+  public Model<Observation>[] getModels() {
+    Model<Observation>[] result = new Model[numClusters];
+    for (int i = 0; i < numClusters; i++)
+      result[i] = clusters.get(i).model;
+    return result;
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonDirichletStateAdapter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonDirichletStateAdapter.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonDirichletStateAdapter.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonDirichletStateAdapter.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,88 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.lang.reflect.Type;
+import java.util.List;
+
+import org.apache.mahout.clustering.dirichlet.models.Model;
+import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
+import org.apache.mahout.matrix.Vector;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+import com.google.gson.reflect.TypeToken;
+
+@SuppressWarnings("unchecked")
+public class JsonDirichletStateAdapter implements
+    JsonSerializer<DirichletState>, JsonDeserializer<DirichletState> {
+
+  Type typeOfModel = new TypeToken<List<DirichletCluster<Vector>>>() {
+  }.getType();
+
+  Type typeOfModelDistribution = new TypeToken<ModelDistribution<Vector>>() {
+  }.getType();
+
+  public JsonElement serialize(DirichletState src, Type typeOfSrc,
+      JsonSerializationContext context) {
+    GsonBuilder builder = new GsonBuilder();
+    builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
+    builder.registerTypeAdapter(Model.class, new JsonModelAdapter());
+    builder.registerTypeAdapter(ModelDistribution.class,
+        new JsonModelDistributionAdapter());
+    Gson gson = builder.create();
+    JsonObject obj = new JsonObject();
+    obj.addProperty("numClusters", src.numClusters);
+    obj.addProperty("offset", src.offset);
+    obj.add("modelFactory", new JsonPrimitive(gson.toJson(src.modelFactory,
+        typeOfModelDistribution)));
+    obj.add("clusters", new JsonPrimitive(gson
+        .toJson(src.clusters, typeOfModel)));
+    obj.add("mixture",
+        new JsonPrimitive(gson.toJson(src.mixture, Vector.class)));
+    return obj;
+  }
+
+  public DirichletState deserialize(JsonElement json, Type typeOfT,
+      JsonDeserializationContext context) throws JsonParseException {
+    GsonBuilder builder = new GsonBuilder();
+    builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
+    builder.registerTypeAdapter(Model.class, new JsonModelAdapter());
+    builder.registerTypeAdapter(ModelDistribution.class,
+        new JsonModelDistributionAdapter());
+    Gson gson = builder.create();
+    JsonObject obj = json.getAsJsonObject();
+    DirichletState state = new DirichletState();
+    state.numClusters = obj.get("numClusters").getAsInt();
+    state.offset = obj.get("offset").getAsDouble();
+    state.modelFactory = gson.fromJson(obj.get("modelFactory").getAsString(),
+        typeOfModelDistribution);
+    state.clusters = gson.fromJson(obj.get("clusters").getAsString(),
+        typeOfModel);
+    state.mixture = gson.fromJson(obj.get("mixture").getAsString(), Vector.class);
+    return state;
+  }
+
+}

Added: 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=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelAdapter.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,67 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.lang.reflect.Type;
+
+import org.apache.mahout.clustering.dirichlet.models.Model;
+import org.apache.mahout.matrix.Vector;
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+
+@SuppressWarnings("unchecked")
+public class JsonModelAdapter implements JsonSerializer<Model>,
+    JsonDeserializer<Model> {
+
+  public JsonElement serialize(Model src, Type typeOfSrc,
+      JsonSerializationContext context) {
+    GsonBuilder builder = new GsonBuilder();
+    builder.registerTypeAdapter(Vector.class, new JsonVectorAdapter());
+    Gson gson = builder.create();
+    JsonObject obj = new JsonObject();
+    obj.add("class", new JsonPrimitive(src.getClass().getName()));
+    obj.add("model", new JsonPrimitive(gson.toJson(src)));
+    return obj;
+  }
+
+  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();
+    JsonObject obj = json.getAsJsonObject();
+    String klass = obj.get("class").getAsString();
+    String model = obj.get("model").getAsString();
+    ClassLoader ccl = Thread.currentThread().getContextClassLoader();
+    Class<?> cl = null;
+    try {
+      cl = ccl.loadClass(klass);
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+    }
+    return (Model) gson.fromJson(model, cl);
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelDistributionAdapter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelDistributionAdapter.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelDistributionAdapter.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonModelDistributionAdapter.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,59 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.lang.reflect.Type;
+
+import org.apache.mahout.clustering.dirichlet.models.ModelDistribution;
+
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+
+@SuppressWarnings("unchecked")
+public class JsonModelDistributionAdapter implements
+    JsonSerializer<ModelDistribution>, JsonDeserializer<ModelDistribution> {
+
+  public JsonElement serialize(ModelDistribution src, Type typeOfSrc,
+      JsonSerializationContext context) {
+    return new JsonPrimitive(src.getClass().getName());
+  }
+
+  public ModelDistribution deserialize(JsonElement json, Type typeOfT,
+      JsonDeserializationContext context) throws JsonParseException {
+    ClassLoader ccl = Thread.currentThread().getContextClassLoader();
+    Class<?> cl = null;
+    try {
+      cl = ccl.loadClass(json.getAsString());
+    } catch (ClassNotFoundException e) {
+      e.printStackTrace();
+    }
+    try {
+      return (ModelDistribution) cl.newInstance();
+    } catch (InstantiationException e) {
+      e.printStackTrace();
+    } catch (IllegalAccessException e) {
+      e.printStackTrace();
+    }
+    return null;
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonVectorAdapter.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonVectorAdapter.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonVectorAdapter.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/JsonVectorAdapter.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,45 @@
+/**
+ * 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.clustering.dirichlet;
+
+import java.lang.reflect.Type;
+
+import org.apache.mahout.matrix.AbstractVector;
+import org.apache.mahout.matrix.Vector;
+
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParseException;
+import com.google.gson.JsonPrimitive;
+import com.google.gson.JsonSerializationContext;
+import com.google.gson.JsonSerializer;
+
+public class JsonVectorAdapter implements JsonSerializer<Vector>,
+    JsonDeserializer<Vector> {
+
+  public JsonElement serialize(Vector src, Type typeOfSrc,
+      JsonSerializationContext context) {
+    return new JsonPrimitive(src.asFormatString());
+  }
+
+  public Vector deserialize(JsonElement json, Type typeOfT,
+      JsonDeserializationContext context) throws JsonParseException {
+    return AbstractVector.decodeVector(json.getAsJsonPrimitive().getAsString());
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/UncommonDistributions.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/UncommonDistributions.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/UncommonDistributions.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/UncommonDistributions.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,270 @@
+package org.apache.mahout.clustering.dirichlet;
+
+/**
+ * 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.Random;
+
+import org.apache.mahout.matrix.DenseVector;
+import org.apache.mahout.matrix.Vector;
+import org.uncommons.maths.random.GaussianGenerator;
+import org.uncommons.maths.random.MersenneTwisterRNG;
+
+public class UncommonDistributions {
+
+  static final double sqrt2pi = Math.sqrt(2 * Math.PI);
+
+  static Random random = new MersenneTwisterRNG();
+
+  public static void init(byte[] seed) {
+    random = new MersenneTwisterRNG(seed);
+  }
+
+  //=============== start of BSD licensed code. See LICENSE.txt
+  /** 
+   * Returns a double sampled according to this distribution.  Uniformly
+   * fast for all k > 0.  (Reference: Non-Uniform Random Variate Generation,
+   * Devroye http://cgm.cs.mcgill.ca/~luc/rnbookindex.html)  Uses Cheng's
+   * rejection algorithm (GB) for k>=1, rejection from Weibull distribution
+   * for 0 < k < 1.
+   */
+  public static double rGamma(double k, double lambda) {
+    boolean accept = false;
+    if (k >= 1) {
+      //Cheng's algorithm
+      double b = (k - Math.log(4));
+      double c = (k + Math.sqrt(2 * k - 1));
+      double lam = Math.sqrt(2 * k - 1);
+      double cheng = (1 + Math.log(4.5));
+      double u, v, x, y, z, r;
+      do {
+        u = random.nextDouble();
+        v = random.nextDouble();
+        y = ((1 / lam) * Math.log(v / (1 - v)));
+        x = (k * Math.exp(y));
+        z = (u * v * v);
+        r = (b + (c * y) - x);
+        if ((r >= ((4.5 * z) - cheng)) || (r >= Math.log(z))) {
+          accept = true;
+        }
+      } while (!accept);
+      return new Double(x / lambda);
+    } else {
+      //Weibull algorithm
+      double c = (1 / k);
+      double d = ((1 - k) * Math.pow(k, (k / (1 - k))));
+      double u, v, z, e, x;
+      do {
+        u = random.nextDouble();
+        v = random.nextDouble();
+        z = -Math.log(u); //generating random exponential variates
+        e = -Math.log(v);
+        x = Math.pow(z, c);
+        if ((z + e) >= (d + x)) {
+          accept = true;
+        }
+      } while (!accept);
+      return new Double(x / lambda);
+    }
+  }
+
+  //============= end of BSD licensed code
+
+  /**
+   * Returns a random sample from a beta distribution with
+   * the given shapes
+   * 
+   * @param shape1 a double representing shape1
+   * @param shape2 a double representing shape2
+   * @return a Vector of samples
+   */
+  public static double rBeta(double shape1, double shape2) {
+    double gam1 = rGamma(shape1, 1);
+    double gam2 = rGamma(shape2, 1);
+    double result = gam1 / (gam1 + gam2);
+    return result;
+
+  }
+
+  /**
+   * Returns a vector of random samples from a beta distribution with
+   * the given shapes
+   * 
+   * @param K the number of samples to return
+   * @param shape1 a double representing shape1
+   * @param shape2 a double representing shape2
+   * @return a Vector of samples
+   */
+  public static Vector rBeta(int K, double shape1, double shape2) {
+    List<Double> params = new ArrayList<Double>(2);
+    params.add(shape1);
+    params.add(Math.max(0, shape2));
+    Vector result = new DenseVector(K);
+    for (int i = 0; i < K; i++)
+      result.set(i, rBeta(shape1, shape2));
+    return result;
+  }
+
+  /**
+   * Return a random sample from the chi-squared (chi^2) distribution with df 
+   * degrees of freedom.
+   * @param df
+   * @return a double sample
+   */
+  public static double rChisq(double df) {
+    double result = 0;
+    for (int i = 0; i < df; i++) {
+      double sample = rNorm(0, 1);
+      result += sample * sample;
+    }
+    return result;
+  }
+
+  /**
+   * Return a random value from a normal distribution with the given mean and
+   * standard deviation
+   * 
+   * @param mean a double mean value
+   * @param sd a double standard deviation
+   * @return a double sample
+   */
+  public static double rNorm(double mean, double sd) {
+    GaussianGenerator dist = new GaussianGenerator(mean, sd, random);
+    return dist.nextValue();
+  }
+
+  /**
+   * Return the normal density function value for the sample x
+   * 
+   * pdf = 1/[sqrt(2*p)*s] * e^{-1/2*[(x-m)/s]^2}
+   *  
+   * 
+   * @param x a double sample value
+   * @param m a double mean value
+   * @param s a double standard deviation
+   * @return a double probability value
+   */
+  public static double dNorm(double x, double m, double s) {
+    double xms = (x - m) / s;
+    double ex = (xms * xms) / 2;
+    double exp = Math.exp(-ex);
+    double result = exp / (sqrt2pi * s);
+    return result;
+  }
+
+  /**
+  * Returns one sample from a multinomial.
+  */
+  public static int rMultinom(Vector probabilities) {
+    // our probability argument are not normalized.
+    double total = probabilities.zSum();
+    double nextDouble = random.nextDouble();
+    double p = nextDouble * total;
+    for (int i = 0; i < probabilities.cardinality(); i++) {
+      double p_i = probabilities.get(i);
+      if (p < p_i) {
+        return i;
+      } else {
+        p -= p_i;
+      }
+    }
+    // can't happen except for round-off error so we don't care what we return here
+    return 0;
+  }
+
+  /**
+   * Returns a multinomial vector sampled from the given probabilities
+   * 
+   * rmultinom should be implemented as successive binomial sampling.
+   *
+   *Keep a normalizing amount that starts with 1 (I call it total).
+   *
+   * For each i
+   *  k[i] = rbinom(p[i] / total, size);
+   *  total -= p[i];
+   *  size -= k[i];
+   * 
+   * @param size the size parameter of the binomial distribution
+   * @param probabilities a Vector of probabilities
+   * 
+   * @return a multinomial distribution Vector
+   */
+  public static Vector rMultinom(int size, Vector probabilities) {
+    // our probability argument may not be normalized.
+    double total = probabilities.zSum();
+    int cardinality = probabilities.cardinality();
+    Vector result = new DenseVector(cardinality);
+    for (int i = 0; total > 0 && i < cardinality; i++) {
+      double p = probabilities.get(i);
+      int ki = rBinomial(size, p / total);
+      total -= p;
+      size -= ki;
+      result.set(i, ki);
+    }
+    return result;
+  }
+
+  /**
+   * Returns an integer sampled according to this distribution.  Takes time
+   * proprotional to np + 1.  (Reference: Non-Uniform Random Variate 
+   * Generation, Devroye http://cgm.cs.mcgill.ca/~luc/rnbookindex.html) 
+   * Second time-waiting algorithm.
+   */
+  public static int rBinomial(int n, double p) {
+    if (p >= 1)
+      return n; // needed to avoid infinite loops and negative results
+    double q = -Math.log(1 - p);
+    double sum = 0;
+    int x = 0;
+    double u, e;
+    while (sum <= q) {
+      u = random.nextDouble();
+      e = -Math.log(u); //exponential random variate
+      sum += (e / (n - x));
+      x += 1;
+    }
+    if (x == 0)
+      return 0;
+    return x - 1;
+  }
+
+  /**
+   * Sample from a Dirichlet distribution over the given alpha, 
+   * returning a vector of probabilities using a stick-breaking algorithm
+   * 
+   * @param alpha an unnormalized count Vector
+   * @return a Vector of probabilities
+   */
+  public static Vector rDirichlet(Vector alpha) {
+    Vector r = alpha.like();
+    double a;
+    double total = alpha.zSum();
+    double remainder = 1;
+    for (int i = 0; i < r.size(); i++) {
+      a = alpha.get(i);
+      total -= a;
+      double beta = rBeta(a, Math.max(0, total));
+      double p = beta * remainder;
+      r.set(i, p);
+      remainder -= p;
+    }
+    return r;
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalDistribution.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalDistribution.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalDistribution.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalDistribution.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,63 @@
+package org.apache.mahout.clustering.dirichlet.models;
+
+/**
+ * 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 org.apache.mahout.clustering.dirichlet.UncommonDistributions;
+import org.apache.mahout.matrix.DenseVector;
+import org.apache.mahout.matrix.Vector;
+
+/**
+ * An implementation of the ModelDistribution interface suitable for testing the
+ * DirichletCluster algorithm. Uses a Normal Distribution to sample the prior
+ * model values. Model values have a vector standard deviation, allowing assymetrical
+ * regions to be covered by a model.
+ */
+public class AsymmetricSampledNormalDistribution implements
+    ModelDistribution<Vector> {
+
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.ModelDistribution#sampleFromPrior(int)
+   */
+  public Model<Vector>[] sampleFromPrior(int howMany) {
+    Model<Vector>[] result = new AsymmetricSampledNormalModel[howMany];
+    for (int i = 0; i < howMany; i++) {
+      double[] m = { UncommonDistributions.rNorm(0, 1),
+          UncommonDistributions.rNorm(0, 1) };
+      DenseVector mean = new DenseVector(m);
+      double[] s = { UncommonDistributions.rNorm(1, 1),
+          UncommonDistributions.rNorm(1, 1) };
+      DenseVector sd = new DenseVector(s);
+      result[i] = new AsymmetricSampledNormalModel(mean, sd);
+    }
+    return result;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.ModelDistribution#sampleFromPosterior(org.apache.mahout.clustering.dirichlet.Model<Observation>[])
+   */
+  public Model<Vector>[] sampleFromPosterior(Model<Vector>[] posterior) {
+    Model<Vector>[] result = new AsymmetricSampledNormalModel[posterior.length];
+    for (int i = 0; i < posterior.length; i++) {
+      AsymmetricSampledNormalModel m = (AsymmetricSampledNormalModel) posterior[i];
+      result[i] = m.sample();
+    }
+    return result;
+  }
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalModel.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalModel.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalModel.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/AsymmetricSampledNormalModel.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,137 @@
+/**
+ * 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.clustering.dirichlet.models;
+
+import org.apache.mahout.matrix.Vector;
+
+public class AsymmetricSampledNormalModel implements Model<Vector> {
+  // the parameters
+  public Vector mean;
+
+  public Vector sd;
+
+  // the observation statistics, initialized by the first observation
+  int s0 = 0;
+
+  Vector s1;
+
+  Vector s2;
+
+  public AsymmetricSampledNormalModel() {
+    super();
+  }
+
+  public AsymmetricSampledNormalModel(Vector mean, Vector sd) {
+    super();
+    this.mean = mean;
+    this.sd = sd;
+    this.s0 = 0;
+    this.s1 = mean.like();
+    this.s2 = mean.like();
+  }
+
+  /**
+   * Return an instance with the same parameters
+   * @return an AsymmetricSampledNormalModel
+   */
+  AsymmetricSampledNormalModel sample() {
+    return new AsymmetricSampledNormalModel(mean, sd);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#observe(java.lang.Object)
+   */
+  public void observe(Vector x) {
+    s0++;
+    if (s1 == null)
+      s1 = x.like();
+    else
+      s1 = s1.plus(x);
+    if (s2 == null)
+      s2 = x.times(x);
+    else
+      s2 = s2.plus(x.times(x));
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#computeParameters()
+   */
+  public void computeParameters() {
+    if (s0 == 0)
+      return;
+    mean = s1.divide(s0);
+    // the average of the two component stds
+    Vector ss = s2.times(s0).minus(s1.times(s1));
+    if (s0 > 1) {
+      sd.set(0, Math.sqrt(ss.get(0)) / s0);
+      sd.set(1, Math.sqrt(ss.get(1)) / s0);
+    } else {
+      sd.set(0, Double.MIN_NORMAL);
+      sd.set(1, Double.MIN_NORMAL);
+    }
+  }
+
+  /**
+   * Calculate a pdf using the supplied sample and sd
+   * 
+  * @param x a Vector sample
+  * @param sd a double std deviation
+  * @return
+  */
+  private double pdf(Vector x, double sd) {
+    assert x.size() == 2;
+    double sd2 = sd * sd;
+    double exp = -(x.dot(x) - 2 * x.dot(mean) + mean.dot(mean)) / (2 * sd2);
+    double ex = Math.exp(exp);
+    double pdf = ex / (sd * Math.sqrt(2 * Math.PI));
+    return pdf;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#pdf(java.lang.Object)
+   */
+  public double pdf(Vector x) {
+    // return the product of the two component pdfs
+    assert x.size() == 2;
+    double pdf0 = pdf(x, sd.get(0));
+    double pdf1 = pdf(x, sd.get(1));
+    if (pdf0 < 0 || pdf0 > 1 || pdf1 < 0 || pdf1 > 1)
+      System.out.print("");
+    return pdf0 * pdf1;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#count()
+   */
+  public int count() {
+    return s0;
+  }
+
+  public String toString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("asnm{n=").append(s0).append(" m=[");
+    if (mean != null)
+      for (int i = 0; i < mean.cardinality(); i++)
+        buf.append(String.format("%.2f", mean.get(i))).append(", ");
+    buf.append("] sd=[");
+    if (sd != null)
+      for (int i = 0; i < sd.cardinality(); i++)
+        buf.append(String.format("%.2f", sd.get(i))).append(", ");
+    buf.append("]}");
+    return buf.toString();
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/Model.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/Model.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/Model.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/Model.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,53 @@
+package org.apache.mahout.clustering.dirichlet.models;
+
+/**
+ * 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.
+ */
+
+/**
+ * A model is a probability distribution over observed data points and allows 
+ * the probability of any data point to be computed.
+ */
+public interface Model<Observation> {
+
+  /**
+   * Observe the given observation, retaining information about it
+   * 
+   * @param x an Observation from the posterior
+   */
+  public abstract void observe(Observation x);
+
+  /**
+   * Compute a new set of posterior parameters based upon the Observations 
+   * that have been observed since my creation
+   */
+  public abstract void computeParameters();
+
+  /**
+  * Return the probability that the observation is described by this model
+  * 
+  * @param x an Observation from the posterior
+  * @return the probability that x is in the receiver
+  */
+  public abstract double pdf(Observation x);
+
+  /**
+   * Return the number of observations that have been observed by this model
+   * 
+   * @return an int
+   */
+  public abstract int count();
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/ModelDistribution.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/ModelDistribution.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/ModelDistribution.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/ModelDistribution.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,42 @@
+package org.apache.mahout.clustering.dirichlet.models;
+
+
+/**
+ * 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.
+ */
+
+/**
+ * A model distribution allows us to sample a model from its prior distribution.
+ */
+public interface ModelDistribution<Observation> {
+
+  /**
+   * Return a list of models sampled from the prior
+   * 
+   * @param howMany the int number of models to return
+   * @return a Model<Observation>[] representing what is known apriori
+   */
+  public Model<Observation>[] sampleFromPrior(int howMany);
+
+  /**
+   * Return a list of models sampled from the posterior
+   * 
+   * @param posterior the Model<Observation>[] after observations
+   * @return a Model<Observation>[] representing what is known apriori
+   */
+  public Model<Observation>[] sampleFromPosterior(Model<Observation>[] posterior);
+
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModel.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModel.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModel.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModel.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,112 @@
+/**
+ * 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.clustering.dirichlet.models;
+
+import org.apache.mahout.matrix.Vector;
+
+public class NormalModel implements Model<Vector> {
+
+  // the parameters
+  public Vector mean;
+
+  public double sd;
+
+  // the observation statistics, initialized by the first observation
+  int s0 = 0;
+
+  Vector s1;
+
+  Vector s2;
+
+  public NormalModel() {
+    super();
+  }
+
+  public NormalModel(Vector mean, double sd) {
+    super();
+    this.mean = mean;
+    this.sd = sd;
+    this.s0 = 0;
+    this.s1 = mean.like();
+    this.s2 = mean.like();
+  }
+
+  /**
+   * Return an instance with the same parameters
+   * @return an NormalModel
+   */
+  NormalModel sample() {
+    return new NormalModel(mean, sd);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#observe(java.lang.Object)
+   */
+  public void observe(Vector x) {
+    s0++;
+    if (s1 == null)
+      s1 = x;
+    else
+      s1 = s1.plus(x);
+    if (s2 == null)
+      s2 = x.times(x);
+    else
+      s2 = s2.plus(x.times(x));
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#computeParameters()
+   */
+  public void computeParameters() {
+    if (s0 == 0)
+      return;
+    mean = s1.divide(s0);
+    // the average of the two component stds
+    if (s0 > 1)
+      sd = Math.sqrt(s2.times(s0).minus(s1.times(s1)).zSum() / 2) / s0;
+    else
+      sd = Double.MIN_VALUE;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#pdf(java.lang.Object)
+   */
+  public double pdf(Vector x) {
+    assert x.size() == 2;
+    double sd2 = sd * sd;
+    double exp = -(x.dot(x) - 2 * x.dot(mean) + mean.dot(mean)) / (2 * sd2);
+    double ex = Math.exp(exp);
+    return ex / (sd * Math.sqrt(2 * Math.PI));
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.Model#count()
+   */
+  public int count() {
+    return s0;
+  }
+
+  public String toString() {
+    StringBuilder buf = new StringBuilder();
+    buf.append("nm{n=").append(s0).append(" m=[");
+    if (mean != null)
+      for (int i = 0; i < mean.cardinality(); i++)
+        buf.append(String.format("%.2f", mean.get(i))).append(", ");
+    buf.append("] sd=").append(String.format("%.2f", sd)).append("}");
+    return buf.toString();
+  }
+}

Added: lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModelDistribution.java
URL: http://svn.apache.org/viewvc/lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModelDistribution.java?rev=754797&view=auto
==============================================================================
--- lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModelDistribution.java (added)
+++ lucene/mahout/trunk/core/src/main/java/org/apache/mahout/clustering/dirichlet/models/NormalModelDistribution.java Mon Mar 16 00:17:07 2009
@@ -0,0 +1,55 @@
+package org.apache.mahout.clustering.dirichlet.models;
+
+/**
+ * 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 org.apache.mahout.matrix.DenseVector;
+import org.apache.mahout.matrix.Vector;
+
+/**
+ * An implementation of the ModelDistribution interface suitable for testing the
+ * DirichletCluster algorithm. Uses a Normal Distribution
+ */
+public class NormalModelDistribution implements ModelDistribution<Vector> {
+
+  public NormalModelDistribution() {
+    super();
+  }
+
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.ModelDistribution#sampleFromPrior(int)
+   */
+  public Model<Vector>[] sampleFromPrior(int howMany) {
+    Model<Vector>[] result = new NormalModel[howMany];
+    for (int i = 0; i < howMany; i++)
+      result[i] = new NormalModel(new DenseVector(2), 1);
+    return result;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.mahout.clustering.dirichlet.ModelDistribution#sampleFromPosterior(org.apache.mahout.clustering.dirichlet.Model<Observation>[])
+   */
+  public Model<Vector>[] sampleFromPosterior(Model<Vector>[] posterior) {
+    Model<Vector>[] result = new NormalModel[posterior.length];
+    for (int i = 0; i < posterior.length; i++) {
+      NormalModel m = (NormalModel) posterior[i];
+      result[i] = m.sample();
+    }
+    return result;
+  }
+}