hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r540733 - in /lucene/hadoop/trunk: ./ src/examples/org/apache/hadoop/examples/ src/examples/org/apache/hadoop/examples/dancing/
Date Tue, 22 May 2007 20:11:01 GMT
Author: cutting
Date: Tue May 22 13:11:00 2007
New Revision: 540733

URL: http://svn.apache.org/viewvc?view=rev&rev=540733
Log:
HADOOP-1413.  Add example program that uses Knuth's dancing links algorithm to solve pentomino problems.  Contributed by Owen.

Added:
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DancingLinks.java
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/OneSidedPentomino.java
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Pentomino.java
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Sudoku.java
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/package.html
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/puzzle1.dta
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=540733&r1=540732&r2=540733
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Tue May 22 13:11:00 2007
@@ -57,6 +57,9 @@
  18. HADOOP-1394.  Implement "performance" code-cleanups in HDFS
      suggested by findbugs.  (Raghu Angadi via cutting)
 
+ 19. HADOOP-1413.  Add example program that uses Knuth's dancing links
+     algorithm to solve pentomino problems.  (omalley via cutting)
+
 
 Branch 0.13 (unreleased changes)
 

Modified: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java?view=diff&rev=540733&r1=540732&r2=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java (original)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/ExampleDriver.java Tue May 22 13:11:00 2007
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.examples;
 import org.apache.hadoop.util.ProgramDriver;
+import org.apache.hadoop.examples.dancing.*;
 
 /**
  * A description of an example program based on its class and a 
@@ -37,6 +38,9 @@
                    "A map/reduce program that writes 10GB of random data per node.");
       pgd.addClass("sort", Sort.class, "A map/reduce program that sorts the data written by the random writer.");
       pgd.addClass("pi", PiEstimator.class, "A map/reduce program that estimates Pi using monte-carlo method.");
+      pgd.addClass("pentomino", DistributedPentomino.class,
+      "A map/reduce tile laying program to find solutions to pentomino problems.");
+      pgd.addClass("sudoku", Sudoku.class, "A sudoku solver.");
       pgd.driver(argv);
     }
     catch(Throwable e){

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DancingLinks.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DancingLinks.java?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DancingLinks.java (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DancingLinks.java Tue May 22 13:11:00 2007
@@ -0,0 +1,437 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.hadoop.examples.dancing;
+
+import java.util.*;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * A generic solver for tile laying problems using Knuth's dancing link
+ * algorithm. It provides a very fast backtracking data structure for problems
+ * that can expressed as a sparse boolean matrix where the goal is to select a
+ * subset of the rows such that each column has exactly 1 true in it.
+ * 
+ * The application gives each column a name and each row is named after the
+ * set of columns that it has as true. Solutions are passed back by giving the 
+ * selected rows' names.
+ * 
+ * The type parameter ColumnName is the class of application's column names.
+ */
+public class DancingLinks<ColumnName> {
+  private static final Log LOG = 
+    LogFactory.getLog(DancingLinks.class.getName());
+  
+  /**
+   * A cell in the table with up/down and left/right links that form doubly
+   * linked lists in both directions. It also includes a link to the column
+   * head.
+   */
+  private static class Node<ColumnName> {
+    Node<ColumnName> left;
+    Node<ColumnName> right;
+    Node<ColumnName> up;
+    Node<ColumnName> down;
+    ColumnHeader<ColumnName> head;
+    
+    Node(Node<ColumnName> l, Node<ColumnName> r, Node<ColumnName> u, 
+         Node<ColumnName> d, ColumnHeader<ColumnName> h) {
+      left = l;
+      right = r;
+      up = u;
+      down = d;
+      head = h;
+    }
+    
+    Node() {
+      this(null, null, null, null, null);
+    }
+  }
+  
+  /**
+   * Column headers record the name of the column and the number of rows that 
+   * satisfy this column. The names are provided by the application and can 
+   * be anything. The size is used for the heuristic for picking the next 
+   * column to explore.
+   */
+  private static class ColumnHeader<ColumnName> extends Node<ColumnName> {
+    ColumnName name;
+    int size;
+    
+    ColumnHeader(ColumnName n, int s) {
+      name = n;
+      size = s;
+      head = this;
+    }
+    
+    ColumnHeader() {
+      this(null, 0);
+    }
+  }
+  
+  /**
+   * The head of the table. Left/Right from the head are the unsatisfied 
+   * ColumnHeader objects.
+   */
+  private ColumnHeader<ColumnName> head;
+  
+  /**
+   * The complete list of columns.
+   */
+  private List<ColumnHeader<ColumnName>> columns;
+  
+  public DancingLinks() {
+    head = new ColumnHeader<ColumnName>(null, 0);
+    head.left = head;
+    head.right = head;
+    head.up = head;
+    head.down = head;
+    columns = new ArrayList<ColumnHeader<ColumnName>>(200);
+  }
+  
+  /**
+   * Add a column to the table
+   * @param name The name of the column, which will be returned as part of 
+   *             solutions
+   * @param primary Is the column required for a solution?
+   */
+  public void addColumn(ColumnName name, boolean primary) {
+    ColumnHeader<ColumnName> top = new ColumnHeader<ColumnName>(name, 0);
+    top.up = top;
+    top.down = top;
+    if (primary) {
+      Node<ColumnName> tail = head.left;
+      tail.right = top;
+      top.left = tail;
+      top.right = head;
+      head.left = top;
+    } else {
+      top.left = top;
+      top.right = top;
+    }
+    columns.add(top);
+  }
+  
+  /**
+   * Add a column to the table
+   * @param name The name of the column, which will be included in the solution
+   */
+  public void addColumn(ColumnName name) {
+    addColumn(name, true);
+  }
+  
+  /**
+   * Get the number of columns.
+   * @return the number of columns
+   */
+  public int getNumberColumns() {
+    return columns.size();
+  }
+  
+  /**
+   * Get the name of a given column as a string
+   * @param index the index of the column
+   * @return a string representation of the name
+   */
+  public String getColumnName(int index) {
+    return columns.get(index).name.toString();
+  }
+  
+  /**
+   * Add a row to the table. 
+   * @param values the columns that are satisfied by this row
+   */
+  public void addRow(boolean[] values) {
+    Node<ColumnName> prev = null;
+    for(int i=0; i < values.length; ++i) {
+      if (values[i]) {
+        ColumnHeader<ColumnName> top = columns.get(i);
+        top.size += 1;
+        Node<ColumnName> bottom = top.up;
+        Node<ColumnName> node = new Node<ColumnName>(null, null, bottom, 
+                                                     top, top);
+        bottom.down = node;
+        top.up = node;
+        if (prev != null) {
+          Node<ColumnName> front = prev.right;
+          node.left = prev;
+          node.right = front;
+          prev.right = node;
+          front.left = node;
+        } else {
+          node.left = node;
+          node.right = node;
+        }
+        prev = node;
+      }
+    }
+  }
+  
+  /**
+   * Applications should implement this to receive the solutions to their 
+   * problems.
+   */
+  public interface SolutionAcceptor<ColumnName> {
+    /**
+     * A callback to return a solution to the application.
+     * @param value a List of List of ColumnNames that were satisfied by each
+     *              selected row
+     */
+    void solution(List<List<ColumnName>> value);
+  }
+  
+  /**
+   * Find the column with the fewest choices.
+   * @return The column header
+   */
+  private ColumnHeader<ColumnName> findBestColumn() {
+    int lowSize = Integer.MAX_VALUE;
+    ColumnHeader<ColumnName> result = null;
+    ColumnHeader<ColumnName> current = (ColumnHeader<ColumnName>) head.right;
+    while (current != head) {
+      if (current.size < lowSize) {
+        lowSize = current.size;
+        result = current;
+      }
+      current = (ColumnHeader<ColumnName>) current.right;
+    }
+    return result;
+  }
+  
+  /**
+   * Hide a column in the table
+   * @param col the column to hide
+   */
+  private void coverColumn(ColumnHeader<ColumnName> col) {
+    LOG.debug("cover " + col.head.name);
+    // remove the column
+    col.right.left = col.left;
+    col.left.right = col.right;
+    Node<ColumnName> row = col.down;
+    while (row != col) {
+      Node<ColumnName> node = row.right;
+      while (node != row) {
+        node.down.up = node.up;
+        node.up.down = node.down;
+        node.head.size -= 1;
+        node = node.right;
+      }
+      row = row.down;
+    }
+  }
+  
+  /**
+   * Uncover a column that was hidden.
+   * @param col the column to unhide
+   */
+  private void uncoverColumn(ColumnHeader<ColumnName> col) {
+    LOG.debug("uncover " + col.head.name);
+    Node<ColumnName> row = col.up;
+    while (row != col) {
+      Node<ColumnName> node = row.left;
+      while (node != row) {
+        node.head.size += 1;
+        node.down.up = node;
+        node.up.down = node;
+        node = node.left;
+      }
+      row = row.up;
+    }
+    col.right.left = col;
+    col.left.right = col;
+  }
+  
+  /**
+   * Get the name of a row by getting the list of column names that it 
+   * satisfies.
+   * @param row the row to make a name for
+   * @return the list of column names
+   */
+  private List<ColumnName> getRowName(Node<ColumnName> row) {
+    List<ColumnName> result = new ArrayList<ColumnName>();
+    result.add(row.head.name);
+    Node<ColumnName> node = row.right;
+    while (node != row) {
+      result.add(node.head.name);
+      node = node.right;
+    }
+    return result;
+  }
+  
+  /**
+   * Find a solution the the problem.
+   * @param partial a temporary datastructure to keep the current partial 
+   *                answer in
+   * @param output the acceptor for the results that are found
+   * @return the number of solutions found
+   */
+  private int search(List<Node<ColumnName>> partial, SolutionAcceptor<ColumnName> output) {
+    int results = 0;
+    if (head.right == head) {
+      List<List<ColumnName>> result = new ArrayList<List<ColumnName>>(partial.size());
+      for(Node<ColumnName> row: partial) {
+        result.add(getRowName(row));
+      }
+      output.solution(result);
+      results += 1;
+    } else {
+      ColumnHeader<ColumnName> col = findBestColumn();
+      if (col.size > 0) {
+        coverColumn(col);
+        Node<ColumnName> row = col.down;
+        while (row != col) {
+          partial.add(row);
+          Node<ColumnName> node = row.right;
+          while (node != row) {
+            coverColumn(node.head);
+            node = node.right;
+          }
+          results += search(partial, output);
+          partial.remove(partial.size() - 1);
+          node = row.left;
+          while (node != row) {
+            uncoverColumn(node.head);
+            node = node.left;
+          }
+          row = row.down;
+        }
+        uncoverColumn(col);
+      }
+    }
+    return results;
+  }
+  
+  /**
+   * Generate a list of prefixes down to a given depth. Assumes that the 
+   * problem is always deeper than depth.
+   * @param depth the depth to explore down
+   * @param choices an array of length depth to describe a prefix
+   * @param prefixes a working datastructure
+   */
+  private void searchPrefixes(int depth, int[] choices, 
+                              List<int[]> prefixes) {
+    if (depth == 0) {
+      prefixes.add(choices.clone());
+    } else {
+      ColumnHeader<ColumnName> col = findBestColumn();
+      if (col.size > 0) {
+        coverColumn(col);
+        Node<ColumnName> row = col.down;
+        int rowId = 0;
+        while (row != col) {
+          Node<ColumnName> node = row.right;
+          while (node != row) {
+            coverColumn(node.head);
+            node = node.right;
+          }
+          choices[choices.length - depth] = rowId;
+          searchPrefixes(depth - 1, choices, prefixes);
+          node = row.left;
+          while (node != row) {
+            uncoverColumn(node.head);
+            node = node.left;
+          }
+          row = row.down;
+          rowId += 1;
+        }
+        uncoverColumn(col);
+      }
+    }
+  }
+  
+  /**
+   * Generate a list of row choices to cover the first moves.
+   * @param depth the length of the prefixes to generate
+   * @return a list of integer arrays that list the rows to pick in order
+   */
+  public List<int[]> split(int depth) {
+    int[] choices = new int[depth];
+    List<int[]> result = new ArrayList<int[]>(100000);
+    searchPrefixes(depth, choices, result);
+    return result;
+  }
+
+  /**
+   * Make one move from a prefix
+   * @param goalRow the row that should be choosen
+   * @return the row that was found
+   */
+  private Node<ColumnName> advance(int goalRow) {
+    ColumnHeader<ColumnName> col = findBestColumn();
+    if (col.size > 0) {
+      coverColumn(col);
+      Node<ColumnName> row = col.down;
+      int id = 0;
+      while (row != col) {
+        if (id == goalRow) {
+          Node<ColumnName> node = row.right;
+          while (node != row) {
+            coverColumn(node.head);
+            node = node.right;
+          }
+          return row;
+        }
+        id += 1;
+        row = row.down;
+      }
+    }
+    return null;
+  }
+  
+  /**
+   * Undo a prefix exploration
+   * @param row
+   */
+  private void rollback(Node<ColumnName> row) {
+    Node<ColumnName> node = row.left;
+    while (node != row) {
+      uncoverColumn(node.head);
+      node = node.left;
+    }
+    uncoverColumn(row.head);
+  }
+  
+  /**
+   * Given a prefix, find solutions under it.
+   * @param prefix a list of row choices that control which part of the search
+   *               tree to explore
+   * @param output the output for each solution
+   * @return the number of solutions
+   */
+  public int solve(int[] prefix, SolutionAcceptor<ColumnName> output) {
+    List<Node<ColumnName>> choices = new ArrayList<Node<ColumnName>>();
+    for(int i=0; i < prefix.length; ++i) {
+      choices.add(advance(prefix[i]));
+    }
+    int result = search(choices, output);
+    for(int i=prefix.length-1; i >=0; --i) {
+      rollback(choices.get(i));
+    }
+    return result;
+  }
+  
+  /**
+   * Solve a complete problem
+   * @param output the acceptor to receive answers
+   * @return the number of solutions
+   */
+  public int solve(SolutionAcceptor<ColumnName> output) {
+    return search(new ArrayList<Node<ColumnName>>(), output);
+  }
+  
+}

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/DistributedPentomino.java Tue May 22 13:11:00 2007
@@ -0,0 +1,198 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.hadoop.examples.dancing;
+
+import java.io.*;
+import java.util.*;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.util.*;
+
+/**
+ * Launch a distributed pentomino solver.
+ * It generates a complete list of prefixes of length N with each unique prefix
+ * as a separate line. A prefix is a sequence of N integers that denote the 
+ * index of the row that is choosen for each column in order. Note that the
+ * next column is heuristically choosen by the solver, so it is dependant on
+ * the previous choice. That file is given as the input to
+ * map/reduce. The output key/value are the move prefix/solution as Text/Text.
+ */
+public class DistributedPentomino {
+
+  /**
+   * Each map takes a line, which represents a prefix move and finds all of 
+   * the solutions that start with that prefix. The output is the prefix as
+   * the key and the solution as the value.
+   */
+  public static class PentMap extends MapReduceBase implements Mapper {
+    private int width;
+    private int height;
+    private int depth;
+    private Pentomino pent;
+    private Text prefixString;
+    private OutputCollector output;
+    private Reporter reporter;
+    
+    /**
+     * For each solution, generate the prefix and a string representation
+     * of the solution. The solution starts with a newline, so that the output
+     * looks like:
+     * <prefix>,
+     * <solution>
+     * 
+     */
+    class SolutionCatcher 
+    implements DancingLinks.SolutionAcceptor<Pentomino.ColumnName> {
+      public void solution(List<List<Pentomino.ColumnName>> answer) {
+        String board = Pentomino.stringifySolution(width, height, answer);
+        try {
+          output.collect(prefixString, new Text("\n" + board));
+          reporter.incrCounter(pent.getCategory(answer), 1);
+        } catch (IOException e) {
+          System.err.println(StringUtils.stringifyException(e));
+        }
+      }
+    }
+    
+    /**
+     * Break the prefix string into moves (a sequence of integer row ids that 
+     * will be selected for each column in order). Find all solutions with
+     * that prefix.
+     */
+    public void map(WritableComparable key, Writable value,
+                    OutputCollector output, Reporter reporter
+                    ) throws IOException {
+      this.output = output;
+      this.reporter = reporter;
+      prefixString = (Text) value;
+      StringTokenizer itr = new StringTokenizer(prefixString.toString(), ",");
+      int[] prefix = new int[depth];
+      int idx = 0;
+      while (itr.hasMoreTokens()) {
+        String num = itr.nextToken();
+        prefix[idx++] = Integer.parseInt(num);
+      }
+      pent.solve(prefix);
+    }
+    
+    public void configure(JobConf conf) {
+      depth = conf.getInt("pent.depth", -1);
+      width = conf.getInt("pent.width", -1);
+      height = conf.getInt("pent.height", -1);
+      pent = (Pentomino) 
+        ReflectionUtils.newInstance(conf.getClass("pent.class", 
+                                                  OneSidedPentomino.class), 
+                                    conf);
+      pent.initialize(width, height);
+      pent.setPrinter(new SolutionCatcher());
+    }
+  }
+  
+  /**
+   * Create the input file with all of the possible combinations of the 
+   * given depth.
+   * @param fs the filesystem to write into
+   * @param dir the directory to write the input file into
+   * @param pent the puzzle 
+   * @param depth the depth to explore when generating prefixes
+   */
+  private static void createInputDirectory(FileSystem fs, 
+                                           Path dir,
+                                           Pentomino pent,
+                                           int depth
+                                           ) throws IOException {
+    fs.mkdirs(dir);
+    List<int[]> splits = pent.getSplits(depth);
+    PrintStream file = 
+      new PrintStream(new BufferedOutputStream
+                      (fs.create(new Path(dir, "part1")), 64*1024));
+    for(int[] prefix: splits) {
+      for(int i=0; i < prefix.length; ++i) {
+        if (i != 0) {
+          file.print(',');          
+        }
+        file.print(prefix[i]);
+      }
+      file.print('\n');
+    }
+    file.close();
+  }
+  
+  /**
+   * Launch the solver on 9x10 board and the one sided pentominos.
+   * This takes about 2.5 hours on 20 nodes with 2 cpus/node.
+   * Splits the job into 2000 maps and 1 reduce.
+   */
+  public static void main(String[] args) throws IOException {
+    JobConf conf;
+    int depth = 5;
+    int width = 9;
+    int height = 10;
+    Class pentClass;
+    if (args.length == 0) {
+      System.out.println("pentomino <output> [conf]");
+      return;
+    }
+    if (args.length == 1) {
+      conf = new JobConf();
+      conf.setInt("pent.width", width);
+      conf.setInt("pent.height", height);
+      conf.setInt("pent.depth", depth);
+      pentClass = OneSidedPentomino.class;
+    } else {
+      conf = new JobConf(args[0]);
+      width = conf.getInt("pent.width", width);
+      height = conf.getInt("pent.height", height);
+      depth = conf.getInt("pent.depth", depth);
+      pentClass = conf.getClass("pent.class", OneSidedPentomino.class);
+    }
+    Path output = new Path(args[0]);
+    Path input = new Path(output + "_input");
+    conf.setInputPath(input);
+    conf.setOutputPath(output);
+    conf.setJarByClass(PentMap.class);
+    FileSystem fileSys = FileSystem.get(conf);
+    conf.setJobName("dancingElephant");
+    Pentomino pent = (Pentomino) ReflectionUtils.newInstance(pentClass, conf);
+    pent.initialize(width, height);
+    createInputDirectory(fileSys, input, pent, depth);
+ 
+    // the keys are the prefix strings
+    conf.setOutputKeyClass(Text.class);
+    // the values are puzzle solutions
+    conf.setOutputValueClass(Text.class);
+    
+    conf.setMapperClass(PentMap.class);        
+    conf.setReducerClass(IdentityReducer.class);
+    
+    conf.setNumMapTasks(2000);
+    conf.setNumReduceTasks(1);
+    
+    // Uncomment to run locally in a single process
+    //conf.set("mapred.job.tracker", "local");
+    
+    JobClient.runJob(conf);
+    fileSys.delete(input);
+  }
+
+}

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/OneSidedPentomino.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/OneSidedPentomino.java?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/OneSidedPentomino.java (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/OneSidedPentomino.java Tue May 22 13:11:00 2007
@@ -0,0 +1,68 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.hadoop.examples.dancing;
+
+/**
+ * Of the "normal" 12 pentominos, 6 of them have distinct shapes when flipped.
+ * This class includes both variants of the "flippable" shapes and the
+ * unflippable shapes for a total of 18 pieces. Clearly, the boards must have
+ * 18*5=90 boxes to hold all of the solutions.
+ */
+public class OneSidedPentomino extends Pentomino {
+
+  public OneSidedPentomino() {}
+
+  public OneSidedPentomino(int width, int height) {
+    super(width, height);
+  }
+
+  /**
+   * Define the one sided pieces. The flipped pieces have the same name with
+   * a capital letter.
+   */
+  protected void initializePieces() {
+    pieces.add(new Piece("x", " x /xxx/ x ", false, oneRotation));
+    pieces.add(new Piece("v", "x  /x  /xxx", false, fourRotations));
+    pieces.add(new Piece("t", "xxx/ x / x ", false, fourRotations));
+    pieces.add(new Piece("w", "  x/ xx/xx ", false, fourRotations));
+    pieces.add(new Piece("u", "x x/xxx", false, fourRotations));
+    pieces.add(new Piece("i", "xxxxx", false, twoRotations));
+    pieces.add(new Piece("f", " xx/xx / x ", false, fourRotations));
+    pieces.add(new Piece("p", "xx/xx/x ", false, fourRotations));
+    pieces.add(new Piece("z", "xx / x / xx", false, twoRotations));
+    pieces.add(new Piece("n", "xx  / xxx", false, fourRotations));
+    pieces.add(new Piece("y", "  x /xxxx", false, fourRotations));
+    pieces.add(new Piece("l", "   x/xxxx", false, fourRotations));
+    pieces.add(new Piece("F", "xx / xx/ x ", false, fourRotations));
+    pieces.add(new Piece("P", "xx/xx/ x", false, fourRotations));
+    pieces.add(new Piece("Z", " xx/ x /xx ", false, twoRotations));
+    pieces.add(new Piece("N", "  xx/xxx ", false, fourRotations));
+    pieces.add(new Piece("Y", " x  /xxxx", false, fourRotations));
+    pieces.add(new Piece("L", "x   /xxxx", false, fourRotations));
+  }
+  
+  /**
+   * Solve the 3x30 puzzle.
+   * @param args
+   */
+  public static void main(String[] args) {
+    Pentomino model = new OneSidedPentomino(3, 30);
+    int solutions = model.solve();
+    System.out.println(solutions + " solutions found.");
+  }
+
+}

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Pentomino.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Pentomino.java?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Pentomino.java (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Pentomino.java Tue May 22 13:11:00 2007
@@ -0,0 +1,448 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.hadoop.examples.dancing;
+
+import java.util.*;
+
+public class Pentomino {
+
+  /**
+   * This interface just is a marker for what types I expect to get back
+   * as column names.
+   */
+  protected static interface ColumnName {
+    // NOTHING
+  }
+
+  /**
+   * Maintain information about a puzzle piece.
+   */
+  protected static class Piece implements ColumnName {
+    private String name;
+    private boolean [][] shape;
+    private int[] rotations;
+    private boolean flippable;
+    
+    public Piece(String name, String shape, 
+                 boolean flippable, int[] rotations) {
+      this.name = name;
+      this.rotations = rotations;
+      this.flippable = flippable;
+      StringTokenizer parser = new StringTokenizer(shape, "/");
+      List<boolean[]> lines = new ArrayList<boolean[]>();
+      while (parser.hasMoreTokens()) {
+        String token = parser.nextToken();
+        boolean[] line = new boolean[token.length()];
+        for(int i=0; i < line.length; ++i) {
+          line[i] = token.charAt(i) == 'x';
+        }
+        lines.add(line);
+      }
+      this.shape = new boolean[lines.size()][];
+      for(int i=0 ; i < lines.size(); i++) {
+        this.shape[i] = (boolean[]) lines.get(i);
+      }
+    }
+    
+    public String getName() {
+      return name;
+    }
+    
+    public int[] getRotations() {
+      return rotations;
+    }
+    
+    public boolean getFlippable() {
+      return flippable;
+    }
+    
+    private int doFlip(boolean flip, int x, int max) {
+      if (flip) {
+        return max - x - 1;
+      } else {
+        return x;
+      }
+    }
+    
+    public boolean[][] getShape(boolean flip, int rotate) {
+      boolean [][] result;
+      if (rotate % 2 == 0) {
+        int height = shape.length;
+        int width = shape[0].length;
+        result = new boolean[height][];
+        boolean flipX = rotate == 2;
+        boolean flipY = flip ^ (rotate == 2);
+        for (int y = 0; y < height; ++y) {
+          result[y] = new boolean[width];
+          for (int x=0; x < width; ++x) {
+            result[y][x] = shape[doFlip(flipY, y, height)]
+                                 [doFlip(flipX, x, width)];
+          }
+        }
+      } else {
+        int height = shape[0].length;
+        int width = shape.length;
+        result = new boolean[height][];
+        boolean flipX = rotate == 3;
+        boolean flipY = flip ^ (rotate == 1);
+        for (int y = 0; y < height; ++y) {
+          result[y] = new boolean[width];
+          for (int x=0; x < width; ++x) {
+            result[y][x] = shape[doFlip(flipX, x, width)]
+                                 [doFlip(flipY, y, height)];
+          }
+        }        
+      }
+      return result;
+    }
+  }
+
+  /**
+   * A point in the puzzle board. This represents a placement of a piece into
+   * a given point on the board.
+   */
+  static class Point implements ColumnName {
+    int x;
+    int y;
+    Point(int x, int y) {
+      this.x = x;
+      this.y = y;
+    }
+  }
+  
+
+  /**
+   * Convert a solution to the puzzle returned by the model into a string
+   * that represents the placement of the pieces onto the board.
+   * @param width the width of the puzzle board
+   * @param height the height of the puzzle board
+   * @param solution the list of column names that were selected in the model
+   * @return a string representation of completed puzzle board
+   */
+  public static String stringifySolution(int width, int height, 
+                                         List<List<ColumnName>> solution) {
+    String[][] picture = new String[height][width];
+    StringBuffer result = new StringBuffer();
+    // for each piece placement...
+    for(List<ColumnName> row: solution) {
+      // go through to find which piece was placed
+      Piece piece = null;
+      for(ColumnName item: row) {
+        if (item instanceof Piece) {
+          piece = (Piece) item;
+          break;
+        }
+      }
+      // for each point where the piece was placed, mark it with the piece name
+      for(ColumnName item: row) {
+        if (item instanceof Point) {
+          Point p = (Point) item;
+          picture[p.y][p.x] = piece.getName();
+        }
+      }
+    }
+    // put the string together
+    for(int y=0; y < picture.length; ++y) {
+      for (int x=0; x < picture[y].length; ++x) {
+        result.append(picture[y][x]);
+      }
+      result.append("\n");
+    }
+    return result.toString();
+  }
+  
+  public enum SolutionCategory {UPPER_LEFT, MID_X, MID_Y, CENTER}
+  
+  /**
+   * Find whether the solution has the x in the upper left quadrant, the
+   * x-midline, the y-midline or in the center.
+   * @param names the solution to check
+   * @return the catagory of the solution
+   */
+  public SolutionCategory getCategory(List<List<ColumnName>> names) {
+    Piece xPiece = null;
+    // find the "x" piece
+    for(Piece p: pieces) {
+      if ("x".equals(p.name)) {
+        xPiece = p;
+        break;
+      }
+    }
+    // find the row containing the "x"
+    for(List<ColumnName> row: names) {
+      if (row.contains(xPiece)) {
+        // figure out where the "x" is located
+        int low_x = width;
+        int high_x = 0;
+        int low_y = height;
+        int high_y = 0;
+        for(ColumnName col: row) {
+          if (col instanceof Point) {
+            int x = ((Point) col).x;
+            int y = ((Point) col).y;
+            if (x < low_x) {
+              low_x = x;
+            }
+            if (x > high_x) {
+              high_x = x;
+            }
+            if (y < low_y) {
+              low_y = y;
+            }
+            if (y > high_y) {
+              high_y = y;
+            }
+          }
+        }
+        boolean mid_x = (low_x + high_x == width - 1);
+        boolean mid_y = (low_y + high_y == height - 1);
+        if (mid_x && mid_y) {
+          return SolutionCategory.CENTER;
+        } else if (mid_x) {
+          return SolutionCategory.MID_X;
+        } else if (mid_y) {
+          return SolutionCategory.MID_Y;
+        }
+        break;
+      }
+    }
+    return SolutionCategory.UPPER_LEFT;
+  }
+  
+  /**
+   * A solution printer that just writes the solution to stdout.
+   */
+  private static class SolutionPrinter 
+                       implements DancingLinks.SolutionAcceptor<ColumnName> {
+    int width;
+    int height;
+    
+    public SolutionPrinter(int width, int height) {
+      this.width = width;
+      this.height = height;
+    }
+    
+    public void solution(List<List<ColumnName>> names) {
+      System.out.println(stringifySolution(width, height, names));
+    }
+  }
+  
+  protected int width;
+  protected int height;
+
+  protected List<Piece> pieces = new ArrayList<Piece>();
+  
+  /**
+   * Is the piece fixed under rotation?
+   */
+  protected static final int [] oneRotation = new int[]{0};
+  
+  /**
+   * Is the piece identical if rotated 180 degrees?
+   */
+  protected static final int [] twoRotations = new int[]{0,1};
+  
+  /**
+   * Are all 4 rotations unique?
+   */
+  protected static final int [] fourRotations = new int[]{0,1,2,3};
+  
+  /**
+   * Fill in the pieces list.
+   */
+  protected void initializePieces() {
+    pieces.add(new Piece("x", " x /xxx/ x ", false, oneRotation));
+    pieces.add(new Piece("v", "x  /x  /xxx", false, fourRotations));
+    pieces.add(new Piece("t", "xxx/ x / x ", false, fourRotations));
+    pieces.add(new Piece("w", "  x/ xx/xx ", false, fourRotations));
+    pieces.add(new Piece("u", "x x/xxx", false, fourRotations));
+    pieces.add(new Piece("i", "xxxxx", false, twoRotations));
+    pieces.add(new Piece("f", " xx/xx / x ", true, fourRotations));
+    pieces.add(new Piece("p", "xx/xx/x ", true, fourRotations));
+    pieces.add(new Piece("z", "xx / x / xx", true, twoRotations));
+    pieces.add(new Piece("n", "xx  / xxx", true, fourRotations));
+    pieces.add(new Piece("y", "  x /xxxx", true, fourRotations));
+    pieces.add(new Piece("l", "   x/xxxx", true, fourRotations));
+  }
+  
+  /**
+   * Is the middle of piece on the upper/left side of the board with 
+   * a given offset and size of the piece? This only checks in one
+   * dimension.
+   * @param offset the offset of the piece
+   * @param shapeSize the size of the piece
+   * @param board the size of the board
+   * @return is it in the upper/left?
+   */
+  private static boolean isSide(int offset, int shapeSize, int board) {
+    return 2*offset + shapeSize <= board;
+  }
+  
+  /**
+   * For a given piece, generate all of the potential placements and add them 
+   * as rows to the model.
+   * @param dancer the problem model
+   * @param piece the piece we are trying to place
+   * @param width the width of the board
+   * @param height the height of the board
+   * @param flip is the piece flipped over?
+   * @param row a workspace the length of the each row in the table
+   * @param upperLeft is the piece constrained to the upper left of the board?
+   *        this is used on a single piece to eliminate most of the trivial
+   *        roations of the solution.
+   */
+  private static void generateRows(DancingLinks dancer,
+                                   Piece piece,
+                                   int width,
+                                   int height,
+                                   boolean flip,
+                                   boolean[] row,
+                                   boolean upperLeft) {
+    // for each rotation
+    int[] rotations = piece.getRotations();
+    for(int rotIndex = 0; rotIndex < rotations.length; ++rotIndex) {
+      // get the shape
+      boolean[][] shape = piece.getShape(flip, rotations[rotIndex]);
+      // find all of the valid offsets
+      for(int x=0; x < width; ++x) {
+        for(int y=0; y < height; ++y) {
+          if (y + shape.length <= height && x + shape[0].length <= width &&
+              (!upperLeft || 
+                  (isSide(x, shape[0].length, width) && 
+                   isSide(y, shape.length, height)))) {
+            // clear the columns related to the points on the board
+            for(int idx=0; idx < width * height; ++idx) {
+              row[idx] = false;
+            }
+            // mark the shape
+            for(int subY=0; subY < shape.length; ++subY) {
+              for(int subX=0; subX < shape[0].length; ++subX) {
+                row[(y + subY) * width + x + subX] = shape[subY][subX];
+              }
+            }
+            dancer.addRow(row);
+          }         
+        }
+      }
+    }
+  }
+  
+  private DancingLinks<ColumnName> dancer = new DancingLinks<ColumnName>();
+  private DancingLinks.SolutionAcceptor<ColumnName> printer;
+  
+  {
+    initializePieces();
+  }
+
+  /**
+   * Create the model for a given pentomino set of pieces and board size.
+   * @param width the width of the board in squares
+   * @param height the height of the board in squares
+   */
+  public Pentomino(int width, int height) {
+    initialize(width, height);
+  }
+
+  /**
+   * Create the object without initialization.
+   */
+  public Pentomino() {
+  }
+
+  void initialize(int width, int height) {
+    this.width = width;
+    this.height = height;
+    for(int y=0; y < height; ++y) {
+      for(int x=0; x < width; ++x) {
+        dancer.addColumn(new Point(x,y));
+      }
+    }
+    int pieceBase = dancer.getNumberColumns();
+    for(Piece p: pieces) {
+      dancer.addColumn(p);
+    }
+    boolean[] row = new boolean[dancer.getNumberColumns()];
+    for(int idx = 0; idx < pieces.size(); ++idx) {
+      Piece piece = (Piece) pieces.get(idx);
+      row[idx + pieceBase] = true;
+      generateRows(dancer, piece, width, height, false, row, idx == 0);
+      if (piece.getFlippable()) {
+        generateRows(dancer, piece, width, height, true, row, idx == 0);
+      }
+      row[idx + pieceBase] = false;
+    }
+    printer = new SolutionPrinter(width, height);
+  }
+  
+  /**
+   * Generate a list of prefixes to a given depth
+   * @param depth the length of each prefix
+   * @return a list of arrays of ints, which are potential prefixes
+   */
+  public List<int[]> getSplits(int depth) {
+    return dancer.split(depth);
+  }
+  
+  /**
+   * Find all of the solutions that start with the given prefix. The printer
+   * is given each solution as it is found.
+   * @param split a list of row indexes that should be choosen for each row
+   *        in order
+   * @return the number of solutions found
+   */
+  public int solve(int[] split) {
+    return dancer.solve(split, printer);
+  }
+  
+  /**
+   * Find all of the solutions to the puzzle.
+   * @return the number of solutions found
+   */
+  public int solve() {
+    return dancer.solve(printer);
+  }
+  
+  /**
+   * Set the printer for the puzzle.
+   * @param printer A call-back object that is given each solution as it is 
+   * found.
+   */
+  public void setPrinter(DancingLinks.SolutionAcceptor<ColumnName> printer) {
+    this.printer = printer;
+  }
+  
+  /**
+   * Solve the 6x10 pentomino puzzle.
+   */
+  public static void main(String[] args) {
+    int width = 6;
+    int height = 10;
+    Pentomino model = new Pentomino(width, height);
+    List splits = model.getSplits(2);
+    for(Iterator splitItr=splits.iterator(); splitItr.hasNext(); ) {
+      int[] choices = (int[]) splitItr.next();
+      System.out.print("split:");
+      for(int i=0; i < choices.length; ++i) {
+        System.out.print(" " + choices[i]);
+      }
+      System.out.println();
+      
+      System.out.println(model.solve(choices) + " solutions found.");
+    }
+  }
+
+}

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Sudoku.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Sudoku.java?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Sudoku.java (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/Sudoku.java Tue May 22 13:11:00 2007
@@ -0,0 +1,316 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.hadoop.examples.dancing;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * This class uses the dancing links algorithm from Knuth to solve sudoku
+ * puzzles. It has solved 42x42 puzzles in 1.02 seconds.
+ */
+public class Sudoku {
+
+  /**
+   * The preset values in the board
+   * board[y][x] is the value at x,y with -1 = any
+   */ 
+  private int[][] board;
+  
+  /**
+   * The size of the board
+   */
+  private int size;
+  
+  /**
+   * The size of the sub-squares in cells across
+   */
+  private int squareXSize;
+  
+  /**
+   * The size of the sub-squares in celss up and down
+   */
+  private int squareYSize;
+  
+  /**
+   * This interface is a marker class for the columns created for the
+   * Sudoku solver.
+   */
+  protected static interface ColumnName {
+    // NOTHING
+  }
+
+  /**
+   * A string containing a representation of the solution.
+   * @param size the size of the board
+   * @param solution a list of list of column names
+   * @return a string of the solution matrix
+   */
+  static String stringifySolution(int size, List<List<ColumnName>> solution) {
+    int[][] picture = new int[size][size];
+    StringBuffer result = new StringBuffer();
+    // go through the rows selected in the model and build a picture of the
+    // solution.
+    for(List<ColumnName> row: solution) {
+      int x = -1;
+      int y = -1;
+      int num = -1;
+      for(ColumnName item: row) {
+        if (item instanceof ColumnConstraint) {
+          x = ((ColumnConstraint) item).column;
+          num = ((ColumnConstraint) item).num;
+        } else if (item instanceof RowConstraint) {
+          y = ((RowConstraint) item).row;
+        }
+      }
+      picture[y][x] = num;
+    }
+    // build the string
+    for(int y=0; y < size; ++y) {
+      for (int x=0; x < size; ++x) {
+        result.append(picture[y][x]);
+        result.append(" ");
+      }
+      result.append("\n");
+    }
+    return result.toString();
+  }
+
+  /**
+   * An acceptor to get the solutions to the puzzle as they are generated and
+   * print them to the console.
+   */
+  private static class SolutionPrinter 
+  implements DancingLinks.SolutionAcceptor<ColumnName> {
+    int size;
+
+    public SolutionPrinter(int size) {
+      this.size = size;
+    }
+
+    /**
+     * A debugging aid that just prints the raw information about the 
+     * dancing link columns that were selected for each row.
+     * @param solution a list of list of column names
+     */
+    void rawWrite(List solution) {
+      for (Iterator itr=solution.iterator(); itr.hasNext(); ) {
+        Iterator subitr = ((List) itr.next()).iterator();
+        while (subitr.hasNext()) {
+          System.out.print(subitr.next().toString() + " ");
+        }
+        System.out.println();
+      }      
+    }
+    
+    public void solution(List<List<ColumnName>> names) {
+      System.out.println(stringifySolution(size, names));
+    }
+  }
+
+  /**
+   * Set up a puzzle board to the given size.
+   * Boards may be asymmetric, but the squares will always be divided to be
+   * more cells wide than they are tall. For example, a 6x6 puzzle will make 
+   * sub-squares that are 3x2 (3 cells wide, 2 cells tall). Clearly that means
+   * the board is made up of 2x3 sub-squares.
+   * @param stream The input stream to read the data from
+   */
+  public Sudoku(InputStream stream) throws IOException {
+    BufferedReader file = new BufferedReader(new InputStreamReader(stream));
+    String line = file.readLine();
+    List<int[]> result = new ArrayList<int[]>();
+    while (line != null) {
+      StringTokenizer tokenizer = new StringTokenizer(line);
+      int size = tokenizer.countTokens();
+      int[] col = new int[size];
+      int y = 0;
+      while(tokenizer.hasMoreElements()) {
+        String word = tokenizer.nextToken();
+        if ("?".equals(word)) {
+          col[y] = - 1;
+        } else {
+          col[y] = Integer.parseInt(word);
+        }
+        y += 1;
+      }
+      result.add(col);
+      line = file.readLine();
+    }
+    size = result.size();
+    board = (int[][]) result.toArray(new int [size][]);
+    squareYSize = (int) Math.sqrt(size);
+    squareXSize = size / squareYSize;
+    file.close();
+  }
+  
+  /**
+   * A constraint that each number can appear just once in a column.
+   */
+  static private class ColumnConstraint implements ColumnName {
+    ColumnConstraint(int num, int column) {
+      this.num = num;
+      this.column = column;
+    }
+    int num;
+    int column;
+    public String toString() {
+      return num + " in column " + column;
+    }
+  }
+  
+  /**
+   * A constraint that each number can appear just once in a row.
+   */
+  static private class RowConstraint implements ColumnName {
+    RowConstraint(int num, int row) {
+      this.num = num;
+      this.row = row;
+    }
+    int num;
+    int row;
+    public String toString() {
+      return num + " in row " + row;
+    }
+  }
+
+  /**
+   * A constraint that each number can appear just once in a square.
+   */
+  static private class SquareConstraint implements ColumnName {
+    SquareConstraint(int num, int x, int y) {
+      this.num = num;
+      this.x = x;
+      this.y = y;
+    }
+    int num;
+    int x;
+    int y;
+    public String toString() {
+      return num + " in square " + x + "," + y;
+    }
+  }
+
+  /**
+   * A constraint that each cell can only be used once.
+   */
+  static private class CellConstraint implements ColumnName {
+    CellConstraint(int x, int y) {
+      this.x = x;
+      this.y = y;
+    }
+    int x;
+    int y;
+    public String toString() {
+      return "cell " + x + "," + y;
+    }
+  }
+  
+  /**
+   * Create a row that places num in cell x, y.
+   * @param rowValues a scratch pad to mark the bits needed
+   * @param x the horizontal offset of the cell
+   * @param y the vertical offset of the cell
+   * @param num the number to place
+   * @return a bitvector of the columns selected
+   */
+  private boolean[] generateRow(boolean[] rowValues, int x, int y, int num) {
+    // clear the scratch array
+    for(int i=0; i < rowValues.length; ++i) {
+      rowValues[i] = false;
+    }
+    // find the square coordinates
+    int xBox = (int) x / squareXSize;
+    int yBox = (int) y / squareYSize;
+    // mark the column
+    rowValues[x*size + num - 1] = true;
+    // mark the row
+    rowValues[size*size + y*size + num - 1] = true;
+    // mark the square
+    rowValues[2*size*size + (xBox*squareXSize + yBox)*size + num - 1] = true;
+    // mark the cell
+    rowValues[3*size*size + size*x + y] = true;
+    return rowValues;
+  }
+  
+  private DancingLinks<ColumnName> makeModel() {
+    DancingLinks<ColumnName> model = new DancingLinks<ColumnName>();
+    // create all of the columns constraints
+    for(int x=0; x < size; ++x) {
+      for(int num=1; num <= size; ++num) {
+        model.addColumn(new ColumnConstraint(num, x));
+      }
+    }
+    // create all of the row constraints
+    for(int y=0; y < size; ++y) {
+      for(int num=1; num <= size; ++num) {
+        model.addColumn(new RowConstraint(num, y));
+      }
+    }
+    // create the square constraints
+    for(int x=0; x < squareYSize; ++x) {
+      for(int y=0; y < squareXSize; ++y) {
+        for(int num=1; num <= size; ++num) {
+          model.addColumn(new SquareConstraint(num, x, y));
+        }
+      }
+    }
+    // create the cell constraints
+    for(int x=0; x < size; ++x) {
+      for(int y=0; y < size; ++y) {
+        model.addColumn(new CellConstraint(x, y));
+      }
+    }
+    boolean[] rowValues = new boolean[size*size*4]; 
+    for(int x=0; x < size; ++x) {
+      for(int y=0; y < size; ++y) {
+        if (board[y][x] == -1) {
+          // try each possible value in the cell
+          for(int num=1; num <= size; ++num) {
+            model.addRow(generateRow(rowValues, x, y, num));
+          }
+        } else {
+          // put the given cell in place
+          model.addRow(generateRow(rowValues, x, y, board[y][x]));
+        }
+      }
+    }
+    return model;
+  }
+  
+  public void solve() {
+    DancingLinks<ColumnName> model = makeModel();
+    int results = model.solve(new SolutionPrinter(size));
+    System.out.println("Found " + results + " solutions");
+  }
+  
+  /**
+   * Solves a set of sudoku puzzles.
+   * @param args a list of puzzle filenames to solve
+   */
+  public static void main(String[] args) throws IOException {
+    if (args.length == 0) {
+      System.out.println("Include a puzzle on the command line.");
+    }
+    for(int i=0; i < args.length; ++i) {
+      Sudoku problem = new Sudoku(new FileInputStream(args[i]));
+      System.out.println("Solving " + args[i]);
+      problem.solve();
+    }
+  }
+
+}

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/package.html
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/package.html?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/package.html (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/package.html Tue May 22 13:11:00 2007
@@ -0,0 +1,57 @@
+<html>
+<body>
+
+This package is a distributed implementation of Knuth's <a
+href="http://en.wikipedia.org/wiki/Dancing_Links>dancing links</a>
+algorithm that can run under Hadoop. It is a generic model for
+problems, such as tile placement, where all of the valid choices can
+be represented as a large sparse boolean array where the goal is to
+pick a subset of the rows to end up with exactly 1 <b>true</b> value
+in each column.
+
+<p>
+
+The package includes two example applications: a <a
+href="http://en.wikipedia.org/wiki/Pentomino">pentomino</a> solver and
+a sudoku solver. 
+
+<p>
+
+The pentomino includes both a "normal" pentomino set and a one-sided
+set where the tiles that are different when flipped are
+duplicated. The pentomino solver has a Hadoop driver application to
+launch it on a cluster. In Knuth's paper on dancing links, he
+describes trying and failing to solve the one-sided pentomino in a
+9x10 board. With the advances of computers and a cluster, it takes a
+small (12 node) hadoop cluster 9 hours to find all of the solutions
+that Knuth estimated would have taken him months.
+
+<p>
+
+The sudoku solver is so fast, I didn't bother making a distributed
+version. (All of the puzzles that I've tried, including a 42x42 have
+taken around a second to solve.) On the command line, give the solver
+a list of puzzle files to solve. Puzzle files have a line per a row
+and columns separated by spaces. The squares either have numbers or
+'?' to mean unknown.
+
+<p>
+
+Both applications have been added to the examples jar, so they can be
+run as:
+
+<pre>
+bin/hadoop jar hadoop-*-examples.jar pentomino pent-outdir
+bin/hadoop jar hadoop-*-examples.jar sudoku puzzle.txt
+</pre>
+
+<p>
+
+I (Owen) implemented the original version of the distributed pentomino
+solver for a Yahoo Hack day, where Yahoos get to work on a project of
+their own choosing for a day to make something cool. The following
+afternoon, everyone gets to show off their hacks and gets a free
+t-shirt. I had a lot of fun doing it.
+
+</body>
+</html>

Added: lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/puzzle1.dta
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/puzzle1.dta?view=auto&rev=540733
==============================================================================
--- lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/puzzle1.dta (added)
+++ lucene/hadoop/trunk/src/examples/org/apache/hadoop/examples/dancing/puzzle1.dta Tue May 22 13:11:00 2007
@@ -0,0 +1,9 @@
+8 5 ? 3 9 ? ? ? ?
+? ? 2 ? ? ? ? ? ?
+? ? 6 ? 1 ? ? ? 2
+? ? 4 ? ? 3 ? 5 9
+? ? 8 9 ? 1 4 ? ?
+3 2 ? 4 ? ? 8 ? ?
+9 ? ? ? 8 ? 5 ? ?
+? ? ? ? ? ? 2 ? ?
+? ? ? ? 4 5 ? 7 8



Mime
View raw message