You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/03/28 12:50:48 UTC

svn commit: r1086181 [13/20] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/solr/ dev-tools/idea/solr/contrib/analysis-extras/ dev-t...

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentSource.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentSource.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ContentSource.java Mon Mar 28 10:50:28 2011
@@ -17,18 +17,11 @@ package org.apache.lucene.benchmark.byTa
  * limitations under the License.
  */
 
-import java.io.BufferedInputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
 
-import org.apache.commons.compress.compressors.CompressorException;
-import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 
 /**
@@ -55,19 +48,6 @@ import org.apache.lucene.benchmark.byTas
  */
 public abstract class ContentSource {
   
-  private static final int BZIP = 0;
-  private static final int GZIP = 1;
-  private static final int OTHER = 2;
-  private static final Map<String,Integer> extensionToType = new HashMap<String,Integer>();
-  static {
-    extensionToType.put(".bz2", Integer.valueOf(BZIP));
-    extensionToType.put(".bzip", Integer.valueOf(BZIP));
-    extensionToType.put(".gz", Integer.valueOf(GZIP));
-    extensionToType.put(".gzip", Integer.valueOf(GZIP));
-  }
-  
-  protected static final int BUFFER_SIZE = 1 << 16; // 64K
-
   private long bytesCount;
   private long totalBytesCount;
   private int docsCount;
@@ -79,8 +59,6 @@ public abstract class ContentSource {
   protected boolean verbose;
   protected String encoding;
   
-  private CompressorStreamFactory csFactory = new CompressorStreamFactory();
-
   /** update count of bytes generated by this source */  
   protected final synchronized void addBytes(long numBytes) {
     bytesCount += numBytes;
@@ -115,51 +93,7 @@ public abstract class ContentSource {
     }
   }
 
-  /**
-   * Returns an {@link InputStream} over the requested file. This method
-   * attempts to identify the appropriate {@link InputStream} instance to return
-   * based on the file name (e.g., if it ends with .bz2 or .bzip, return a
-   * 'bzip' {@link InputStream}).
-   */
-  protected InputStream getInputStream(File file) throws IOException {
-    // First, create a FileInputStream, as this will be required by all types.
-    // Wrap with BufferedInputStream for better performance
-    InputStream is = new BufferedInputStream(new FileInputStream(file), BUFFER_SIZE);
-    
-    String fileName = file.getName();
-    int idx = fileName.lastIndexOf('.');
-    int type = OTHER;
-    if (idx != -1) {
-      Integer typeInt = extensionToType.get(fileName.substring(idx));
-      if (typeInt != null) {
-        type = typeInt.intValue();
-      }
-    }
-    
-    try {
-      switch (type) {
-        case BZIP:
-          // According to BZip2CompressorInputStream's code, it reads the first 
-          // two file header chars ('B' and 'Z'). It is important to wrap the
-          // underlying input stream with a buffered one since
-          // Bzip2CompressorInputStream uses the read() method exclusively.
-          is = csFactory.createCompressorInputStream("bzip2", is);
-          break;
-        case GZIP:
-          is = csFactory.createCompressorInputStream("gz", is);
-          break;
-        default: // Do nothing, stay with FileInputStream
-      }
-    } catch (CompressorException e) {
-      IOException ioe = new IOException(e.getMessage());
-      ioe.initCause(e);
-      throw ioe;
-    }
-    
-    return is;
-  }
-  
-  /**
+	/**
    * Returns true whether it's time to log a message (depending on verbose and
    * the number of documents generated).
    */

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/EnwikiContentSource.java Mon Mar 28 10:50:28 2011
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.benchmark.byTask.utils.StreamUtils;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.xml.sax.Attributes;
 import org.xml.sax.InputSource;
@@ -189,7 +190,7 @@ public class EnwikiContentSource extends
               return;
             } else if (localFileIS == is) {
               // If file is not already re-opened then re-open it now
-              is = getInputStream(file);
+              is = StreamUtils.inputStream(file);
             }
           }
         }
@@ -290,7 +291,7 @@ public class EnwikiContentSource extends
   @Override
   public void resetInputs() throws IOException {
     super.resetInputs();
-    is = getInputStream(file);
+    is = StreamUtils.inputStream(file);
   }
   
   @Override

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/LineDocSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/LineDocSource.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/LineDocSource.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/LineDocSource.java Mon Mar 28 10:50:28 2011
@@ -22,9 +22,13 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.lang.reflect.Constructor;
+import java.util.Arrays;
+import java.util.Properties;
 
 import org.apache.lucene.benchmark.byTask.tasks.WriteLineDocTask;
 import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.benchmark.byTask.utils.StreamUtils;
 
 /**
  * A {@link ContentSource} reading one line at a time as a
@@ -40,23 +44,146 @@ import org.apache.lucene.benchmark.byTas
  * <ul>
  * <li>docs.file=&lt;path to the file&gt;
  * <li>content.source.encoding - default to UTF-8.
+ * <li>line.parser - default to {@link HeaderLineParser} if a header line exists which differs 
+ *     from {@link WriteLineDocTask#DEFAULT_FIELDS} and to {@link SimpleLineParser} otherwise.
  * </ul>
  */
 public class LineDocSource extends ContentSource {
 
-  private final static char SEP = WriteLineDocTask.SEP;
+  /** Reader of a single input line into {@link DocData}. */
+  public static abstract class LineParser {
+    protected final String[] header;
+    /** Construct with the header 
+     * @param header header line found in the input file, or null if none
+     */
+    public LineParser(String[] header) {
+      this.header = header; 
+    }
+    /** parse an input line and fill doc data appropriately */
+    public abstract void parseLine(DocData docData, String line);
+  }
+  
+  /** 
+   * {@link LineParser} which ignores the header passed to its constructor
+   * and assumes simply that field names and their order are the same 
+   * as in {@link WriteLineDocTask#DEFAULT_FIELDS} 
+   */
+  public static class SimpleLineParser extends LineParser {
+    public SimpleLineParser(String[] header) {
+      super(header);
+    }
+    public void parseLine(DocData docData, String line) {
+      int k1 = 0;
+      int k2 = line.indexOf(WriteLineDocTask.SEP, k1);
+      if (k2<0) {
+        throw new RuntimeException("line: [" + line + "] is in an invalid format (missing: separator title::date)!");
+      }
+      docData.setTitle(line.substring(k1,k2));
+      k1 = k2+1;
+      k2 = line.indexOf(WriteLineDocTask.SEP, k1);
+      if (k2<0) {
+        throw new RuntimeException("line: [" + line + "] is in an invalid format (missing: separator date::body)!");
+      }
+      docData.setDate(line.substring(k1,k2));
+      k1 = k2+1;
+      k2 = line.indexOf(WriteLineDocTask.SEP, k1);
+      if (k2>=0) {
+        throw new RuntimeException("line: [" + line + "] is in an invalid format (too many separators)!");
+      }
+      // last one
+      docData.setBody(line.substring(k1));
+    }
+  }
+  
+  /** 
+   * {@link LineParser} which sets field names and order by 
+   * the header - any header - of the lines file.
+   * It is less efficient than {@link SimpleLineParser} but more powerful.
+   */
+  public static class HeaderLineParser extends LineParser {
+    private enum FieldName { NAME , TITLE , DATE , BODY, PROP } 
+    private final FieldName[] posToF;
+    public HeaderLineParser(String[] header) {
+      super(header);
+      posToF = new FieldName[header.length];
+      for (int i=0; i<header.length; i++) {
+        String f = header[i];
+        if (DocMaker.NAME_FIELD.equals(f)) {
+          posToF[i] = FieldName.NAME;
+        } else if (DocMaker.TITLE_FIELD.equals(f)) {
+          posToF[i] = FieldName.TITLE;
+        } else if (DocMaker.DATE_FIELD.equals(f)) {
+          posToF[i] = FieldName.DATE;
+        } else if (DocMaker.BODY_FIELD.equals(f)) {
+          posToF[i] = FieldName.BODY;
+        } else {
+          posToF[i] = FieldName.PROP;
+        }
+      }
+    }
+    
+    public void parseLine(DocData docData, String line) {
+      int n = 0;
+      int k1 = 0;
+      int k2;
+      while ((k2 = line.indexOf(WriteLineDocTask.SEP, k1)) >= 0) {
+        if (n>=header.length) {
+          throw new RuntimeException("input line has invalid format: "+(n+1)+" fields instead of "+header.length+" :: [" + line + "]");
+        }
+        setDocDataField(docData, n, line.substring(k1,k2));
+        ++n;
+        k1 = k2 + 1;
+      }
+      if (n!=header.length-1) {
+        throw new RuntimeException("input line has invalid format: "+(n+1)+" fields instead of "+header.length+" :: [" + line + "]");
+      }
+      // last one
+      setDocDataField(docData, n, line.substring(k1)); 
+    }
 
+    private void setDocDataField(DocData docData, int position, String text) {
+      switch(posToF[position]) {
+        case NAME: 
+          docData.setName(text);
+          break;
+        case TITLE: 
+          docData.setTitle(text);
+          break;
+        case DATE: 
+          docData.setDate(text);
+          break;
+        case BODY: 
+          docData.setBody(text);
+          break;
+        case PROP:
+          Properties p = docData.getProps();
+          if (p==null) {
+            p = new Properties();
+            docData.setProps(p);
+          }
+          p.setProperty(header[position], text);
+          break;
+      }
+    }
+  }
+  
   private File file;
   private BufferedReader reader;
   private int readCount;
 
+  private LineParser docDataLineReader = null;
+  private boolean skipHeaderLine = false;
+
   private synchronized void openFile() {
     try {
       if (reader != null) {
         reader.close();
       }
-      InputStream is = getInputStream(file);
-      reader = new BufferedReader(new InputStreamReader(is, encoding), BUFFER_SIZE);
+      InputStream is = StreamUtils.inputStream(file);
+      reader = new BufferedReader(new InputStreamReader(is, encoding), StreamUtils.BUFFER_SIZE);
+      if (skipHeaderLine) {
+        reader.readLine(); // skip one line - the header line - already handled that info
+      }
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
@@ -77,7 +204,6 @@ public class LineDocSource extends Conte
     
     synchronized(this) {
       line = reader.readLine();
-      myID = readCount++;
       if (line == null) {
         if (!forever) {
           throw new NoMoreDataException();
@@ -86,27 +212,54 @@ public class LineDocSource extends Conte
         openFile();
         return getNextDocData(docData);
       }
+      if (docDataLineReader == null) { // first line ever, one time initialization,
+        docDataLineReader = createDocDataLineReader(line);
+        if (skipHeaderLine) {
+          return getNextDocData(docData);
+        }
+      }
+      // increment IDS only once...
+      myID = readCount++; 
     }
     
-    // A line must be in the following format. If it's not, fail !
-    // title <TAB> date <TAB> body <NEWLINE>
-    int spot = line.indexOf(SEP);
-    if (spot == -1) {
-      throw new RuntimeException("line: [" + line + "] is in an invalid format !");
-    }
-    int spot2 = line.indexOf(SEP, 1 + spot);
-    if (spot2 == -1) {
-      throw new RuntimeException("line: [" + line + "] is in an invalid format !");
-    }
     // The date String was written in the format of DateTools.dateToString.
     docData.clear();
     docData.setID(myID);
-    docData.setBody(line.substring(1 + spot2, line.length()));
-    docData.setTitle(line.substring(0, spot));
-    docData.setDate(line.substring(1 + spot, spot2));
+    docDataLineReader.parseLine(docData, line);
     return docData;
   }
 
+  private LineParser createDocDataLineReader(String line) {
+    String[] header;
+    String headIndicator = WriteLineDocTask.FIELDS_HEADER_INDICATOR + WriteLineDocTask.SEP;
+
+    if (line.startsWith(headIndicator)) {
+      header = line.substring(headIndicator.length()).split(Character.toString(WriteLineDocTask.SEP));
+      skipHeaderLine = true; // mark to skip the header line when input file is reopened
+    } else {
+      header = WriteLineDocTask.DEFAULT_FIELDS;
+    }
+    
+    // if a specific DocDataLineReader was configured, must respect it
+    String docDataLineReaderClassName = getConfig().get("line.parser", null);
+    if (docDataLineReaderClassName!=null) {
+      try {
+        final Class<? extends LineParser> clazz = 
+          Class.forName(docDataLineReaderClassName).asSubclass(LineParser.class);
+        Constructor<? extends LineParser> cnstr = clazz.getConstructor(new Class[]{String[].class});
+        return cnstr.newInstance((Object)header);
+      } catch (Exception e) {
+        throw new RuntimeException("Failed to instantiate "+docDataLineReaderClassName, e);
+      }
+    }
+
+    // if this the simple case,   
+    if (Arrays.deepEquals(header, WriteLineDocTask.DEFAULT_FIELDS)) {
+      return new SimpleLineParser(header);
+    }
+    return new HeaderLineParser(header);
+  }
+
   @Override
   public void resetInputs() throws IOException {
     super.resetInputs();

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecContentSource.java Mon Mar 28 10:50:28 2011
@@ -32,6 +32,7 @@ import java.util.Locale;
 
 import org.apache.lucene.benchmark.byTask.feeds.TrecDocParser.ParsePathType;
 import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.benchmark.byTask.utils.StreamUtils;
 import org.apache.lucene.benchmark.byTask.utils.StringBuilderReader;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -194,8 +195,8 @@ public class TrecContentSource extends C
         System.out.println("opening: " + f + " length: " + f.length());
       }
       try {
-        InputStream inputStream = getInputStream(f); // support either gzip, bzip2, or regular text file, by extension  
-        reader = new BufferedReader(new InputStreamReader(inputStream, encoding), BUFFER_SIZE);
+        InputStream inputStream = StreamUtils.inputStream(f); // support either gzip, bzip2, or regular text file, by extension  
+        reader = new BufferedReader(new InputStreamReader(inputStream, encoding), StreamUtils.BUFFER_SIZE);
         currPathType = TrecDocParser.pathType(f);
         return;
       } catch (Exception e) {

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/package.html?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/package.html (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/package.html Mon Mar 28 10:50:28 2011
@@ -170,6 +170,9 @@ with the benchmark.ext.classpath propert
       <font color="#FF0000">-Dbenchmark.ext.classpath=/mydir/classes
       </font> -Dtask.mem=512M</li>
 </ul>
+<u>External tasks</u>: When writing your own tasks under a package other than 
+<b>org.apache.lucene.benchmark.byTask.tasks</b> specify that package thru the
+<font color="#FF0000">alt.tasks.packages</font> property.
 </p>
 
 <a name="algorithm"></a>
@@ -590,6 +593,14 @@ Here is a list of currently defined prop
     <ul><li>doc.delete.step
     </li></ul>
   </li>
+  
+  <li><b>Task alternative packages</b>:
+    <ul><li>alt.tasks.packages
+      - comma separated list of additional packages where tasks classes will be looked for
+      when not found in the default package (that of PerfTask).  If the same task class 
+      appears in more than one package, the package indicated first in this list will be used.
+    </li></ul> 
+  </li>
 
 </ol>
 

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTask.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTask.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTask.java Mon Mar 28 10:50:28 2011
@@ -17,19 +17,20 @@ package org.apache.lucene.benchmark.byTa
  * limitations under the License.
  */
 
-import java.io.BufferedOutputStream;
 import java.io.BufferedWriter;
-import java.io.FileOutputStream;
+import java.io.File;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
+import java.util.Arrays;
+import java.util.HashSet;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.lucene.benchmark.byTask.PerfRunData;
 import org.apache.lucene.benchmark.byTask.feeds.DocMaker;
 import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.benchmark.byTask.utils.StreamUtils;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 
@@ -38,14 +39,23 @@ import org.apache.lucene.document.Field;
  * following format: title &lt;TAB&gt; date &lt;TAB&gt; body. The output of this
  * task can be consumed by
  * {@link org.apache.lucene.benchmark.byTask.feeds.LineDocSource} and is intended
- * to save the IO overhead of opening a file per document to be indexed.<br>
+ * to save the IO overhead of opening a file per document to be indexed.
+ * <p>
+ * The format of the output is set according to the output file extension.
+ * Compression is recommended when the output file is expected to be large.
+ * See info on file extensions in {@link StreamUtils.Type}
+ * <p> 
  * Supports the following parameters:
  * <ul>
- * <li>line.file.out - the name of the file to write the output to. That
+ * <li><b>line.file.out</b> - the name of the file to write the output to. That
  * parameter is mandatory. <b>NOTE:</b> the file is re-created.
- * <li>bzip.compression - whether the output should be bzip-compressed. This is
- * recommended when the output file is expected to be large. (optional, default:
- * false).
+ * <li><b>line.fields</b> - which fields should be written in each line.
+ * (optional, default: {@link #DEFAULT_FIELDS}).
+ * <li><b>sufficient.fields</b> - list of field names, separated by comma, which, 
+ * if all of them are missing, the document will be skipped. For example, to require 
+ * that at least one of f1,f2 is not empty, specify: "f1,f2" in this field. To specify
+ * that no field is required, i.e. that even empty docs should be emitted, specify <b>","</b>.    
+ * (optional, default: {@link #DEFAULT_SUFFICIENT_FIELDS}).
  * </ul>
  * <b>NOTE:</b> this class is not thread-safe and if used by multiple threads the
  * output is unspecified (as all will write to the same output file in a
@@ -53,42 +63,88 @@ import org.apache.lucene.document.Field;
  */
 public class WriteLineDocTask extends PerfTask {
 
+  public static final String FIELDS_HEADER_INDICATOR = "FIELDS_HEADER_INDICATOR###";
+
   public final static char SEP = '\t';
   
+  /**
+   * Fields to be written by default
+   */
+  public static final String[] DEFAULT_FIELDS = new String[] {
+    DocMaker.TITLE_FIELD,
+    DocMaker.DATE_FIELD,
+    DocMaker.BODY_FIELD,
+  };
+  
+  /**
+   * Default fields which at least one of them is required to not skip the doc.
+   */
+  public static final String DEFAULT_SUFFICIENT_FIELDS = DocMaker.TITLE_FIELD +',' + DocMaker.BODY_FIELD;
+  
   private int docSize = 0;
   private PrintWriter lineFileOut = null;
   private DocMaker docMaker;
   private ThreadLocal<StringBuilder> threadBuffer = new ThreadLocal<StringBuilder>();
   private ThreadLocal<Matcher> threadNormalizer = new ThreadLocal<Matcher>();
+  private final String[] fieldsToWrite;;
+  private final boolean[] sufficientFields;
+  private final boolean checkSufficientFields;
   
   public WriteLineDocTask(PerfRunData runData) throws Exception {
     super(runData);
     Config config = runData.getConfig();
-    String fileName = config.get("line.file.out", null);
-    if (fileName == null) {
+    String fname = config.get("line.file.out", null);
+    if (fname == null) {
       throw new IllegalArgumentException("line.file.out must be set");
     }
-
-    OutputStream out = new FileOutputStream(fileName);
-    boolean doBzipCompression = false;
-    String doBZCompress = config.get("bzip.compression", null);
-    if (doBZCompress != null) {
-      // Property was set, use the value.
-      doBzipCompression = Boolean.valueOf(doBZCompress).booleanValue();
+    OutputStream out = StreamUtils.outputStream(new File(fname));
+    lineFileOut = new PrintWriter(new BufferedWriter(new OutputStreamWriter(out, "UTF-8"), StreamUtils.BUFFER_SIZE));
+    docMaker = runData.getDocMaker();
+    
+    // init fields 
+    String f2r = config.get("line.fields",null);
+    if (f2r == null) {
+      fieldsToWrite = DEFAULT_FIELDS;
+    } else {
+      if (f2r.indexOf(SEP)>=0) {
+        throw new IllegalArgumentException("line.fields "+f2r+" should not contain the separator char: "+SEP);
+      }
+      fieldsToWrite = f2r.split(","); 
+    }
+    
+    // init sufficient fields
+    sufficientFields = new boolean[fieldsToWrite.length];
+    String suff = config.get("sufficient.fields",DEFAULT_SUFFICIENT_FIELDS);
+    if (",".equals(suff)) {
+      checkSufficientFields = false;
     } else {
-      // Property was not set, attempt to detect based on file's extension
-      doBzipCompression = fileName.endsWith("bz2");
+      checkSufficientFields = true;
+      HashSet<String> sf = new HashSet<String>(Arrays.asList(suff.split(",")));
+      for (int i=0; i<fieldsToWrite.length; i++) {
+        if (sf.contains(fieldsToWrite[i])) {
+          sufficientFields[i] = true;
+        }
+      }
     }
+    
+    writeHeader();
+  }
 
-    if (doBzipCompression) {
-      // Wrap with BOS since BZip2CompressorOutputStream calls out.write(int) 
-      // and does not use the write(byte[]) version. This proved to speed the 
-      // compression process by 70% !
-      out = new BufferedOutputStream(out, 1 << 16);
-      out = new CompressorStreamFactory().createCompressorOutputStream("bzip2", out);
+  /**
+   * Write a header to the lines file - indicating how to read the file later 
+   */
+  private void writeHeader() {
+    StringBuilder sb = threadBuffer.get();
+    if (sb == null) {
+      sb = new StringBuilder();
+      threadBuffer.set(sb);
     }
-    lineFileOut = new PrintWriter(new BufferedWriter(new OutputStreamWriter(out, "UTF-8"), 1 << 16));
-    docMaker = runData.getDocMaker();
+    sb.setLength(0);
+    sb.append(FIELDS_HEADER_INDICATOR);
+    for (String f : fieldsToWrite) {
+      sb.append(SEP).append(f);
+    }
+    lineFileOut.println(sb.toString());
   }
 
   @Override
@@ -106,27 +162,26 @@ public class WriteLineDocTask extends Pe
       threadNormalizer.set(matcher);
     }
     
-    Field f = doc.getField(DocMaker.BODY_FIELD);
-    String body = f != null ? matcher.reset(f.stringValue()).replaceAll(" ") : "";
-    
-    f = doc.getField(DocMaker.TITLE_FIELD);
-    String title = f != null ? matcher.reset(f.stringValue()).replaceAll(" ") : "";
-    
-    if (body.length() > 0 || title.length() > 0) {
-      
-      f = doc.getField(DocMaker.DATE_FIELD);
-      String date = f != null ? matcher.reset(f.stringValue()).replaceAll(" ") : "";
-      
-      StringBuilder sb = threadBuffer.get();
-      if (sb == null) {
-        sb = new StringBuilder();
-        threadBuffer.set(sb);
-      }
-      sb.setLength(0);
-      sb.append(title).append(SEP).append(date).append(SEP).append(body);
+    StringBuilder sb = threadBuffer.get();
+    if (sb == null) {
+      sb = new StringBuilder();
+      threadBuffer.set(sb);
+    }
+    sb.setLength(0);
+
+    boolean sufficient = !checkSufficientFields;
+    for (int i=0; i<fieldsToWrite.length; i++) {
+      Field f = doc.getField(fieldsToWrite[i]);
+      String text = f == null ? "" : matcher.reset(f.stringValue()).replaceAll(" ").trim();
+      sb.append(text).append(SEP);
+      sufficient |= text.length()>0 && sufficientFields[i];
+    }
+    if (sufficient) {
+      sb.setLength(sb.length()-1); // remove redundant last separator
       // lineFileOut is a PrintWriter, which synchronizes internally in println.
       lineFileOut.println(sb.toString());
     }
+
     return 1;
   }
 

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Algorithm.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Algorithm.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Algorithm.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/Algorithm.java Mon Mar 28 10:50:28 2011
@@ -21,6 +21,7 @@ import java.io.StreamTokenizer;
 import java.io.StringReader;
 import java.lang.reflect.Constructor;
 import java.util.ArrayList;
+import java.util.Arrays;
 
 import org.apache.lucene.benchmark.byTask.PerfRunData;
 import org.apache.lucene.benchmark.byTask.tasks.PerfTask;
@@ -33,15 +34,23 @@ import org.apache.lucene.benchmark.byTas
 public class Algorithm {
   
   private TaskSequence sequence;
+  private final String[] taskPackages;
   
   /**
    * Read algorithm from file
+   * Property examined: alt.tasks.packages == comma separated list of 
+   * alternate package names where tasks would be searched for, when not found 
+   * in the default package (that of {@link PerfTask}{@link #getClass()}).
+   * If the same task class appears in more than one package, the package 
+   * indicated first in this list will be used.
    * @param runData perf-run-data used at running the tasks.
    * @throws Exception if errors while parsing the algorithm 
    */
   @SuppressWarnings("fallthrough")
   public Algorithm (PerfRunData runData) throws Exception {
-    String algTxt = runData.getConfig().getAlgorithmText();
+    Config config = runData.getConfig();
+    taskPackages = initTasksPackages(config);
+    String algTxt = config.getAlgorithmText();
     sequence = new TaskSequence(runData,null,null,false);
     TaskSequence currSequence = sequence;
     PerfTask prevTask = null;
@@ -55,14 +64,13 @@ public class Algorithm {
     boolean colonOk = false; 
     boolean isDisableCountNextTask = false; // only for primitive tasks
     currSequence.setDepth(0);
-    String taskPackage = PerfTask.class.getPackage().getName() + ".";
     
     while (stok.nextToken() != StreamTokenizer.TT_EOF) { 
       switch(stok.ttype) {
   
         case StreamTokenizer.TT_WORD:
           String s = stok.sval;
-          Constructor<? extends PerfTask> cnstr = Class.forName(taskPackage+s+"Task")
+          Constructor<? extends PerfTask> cnstr = taskClass(config,s)
             .asSubclass(PerfTask.class).getConstructor(PerfRunData.class);
           PerfTask task = cnstr.newInstance(runData);
           task.setDisableCounting(isDisableCountNextTask);
@@ -248,9 +256,33 @@ public class Algorithm {
     }
   }
 
-  /* (non-Javadoc)
-   * @see java.lang.Object#toString()
-   */
+  private String[] initTasksPackages(Config config) {
+    String alts = config.get("alt.tasks.packages", null);
+    String dfltPkg = PerfTask.class.getPackage().getName();
+    if (alts==null) {
+      return new String[]{ dfltPkg };
+    }
+    ArrayList<String> pkgs = new ArrayList<String>();
+    pkgs.add(dfltPkg);
+    for (String alt : alts.split(",")) {
+      pkgs.add(alt);
+    }
+    return pkgs.toArray(new String[0]);
+  }
+
+  private Class<?> taskClass(Config config, String taskName)
+      throws ClassNotFoundException {
+    for (String pkg : taskPackages) {
+      try {
+        return Class.forName(pkg+'.'+taskName+"Task");
+      } catch (ClassNotFoundException e) {
+        // failed in this package, might succeed in the next one... 
+      }
+    }
+    // can only get here if failed to instantiate
+    throw new ClassNotFoundException(taskName+" not found in packages "+Arrays.toString(taskPackages));
+  }
+
   @Override
   public String toString() {
     String newline = System.getProperty("line.separator");

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/QualityQueriesFinder.java Mon Mar 28 10:50:28 2011
@@ -124,7 +124,7 @@ public class QualityQueriesFinder {
   
   private static class TermsDfQueue extends PriorityQueue<TermDf> {
     TermsDfQueue (int maxSize) {
-      initialize(maxSize);
+      super(maxSize);
     }
     @Override
     protected boolean lessThan(TermDf tf1, TermDf tf2) {

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java Mon Mar 28 10:50:28 2011
@@ -36,6 +36,7 @@ import org.apache.lucene.benchmark.byTas
 import org.apache.lucene.benchmark.byTask.stats.TaskStats;
 import org.apache.lucene.benchmark.byTask.tasks.CountingHighlighterTestTask;
 import org.apache.lucene.benchmark.byTask.tasks.CountingSearchTestTask;
+import org.apache.lucene.benchmark.byTask.tasks.WriteLineDocTask;
 import org.apache.lucene.collation.CollationKeyAnalyzer;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldsEnum;
@@ -393,8 +394,13 @@ public class TestPerfTasksLogic extends 
 
     BufferedReader r = new BufferedReader(new FileReader(lineFile));
     int numLines = 0;
-    while(r.readLine() != null)
+    String line;
+    while((line = r.readLine()) != null) {
+      if (numLines==0 && line.startsWith(WriteLineDocTask.FIELDS_HEADER_INDICATOR)) {
+        continue; // do not count the header line as a doc 
+      }
       numLines++;
+    }
     r.close();
     assertEquals("did not see the right number of docs; should be " + NUM_TRY_DOCS + " but was " + numLines, NUM_TRY_DOCS, numLines);
     

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/LineDocSourceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/LineDocSourceTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/LineDocSourceTest.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/LineDocSourceTest.java Mon Mar 28 10:50:28 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.benchmark.byTa
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileOutputStream;
+import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.util.Properties;
@@ -28,6 +29,8 @@ import org.apache.commons.compress.compr
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.benchmark.BenchmarkTestCase;
 import org.apache.lucene.benchmark.byTask.PerfRunData;
+import org.apache.lucene.benchmark.byTask.feeds.LineDocSource.HeaderLineParser;
+import org.apache.lucene.benchmark.byTask.feeds.LineDocSource.LineParser;
 import org.apache.lucene.benchmark.byTask.tasks.AddDocTask;
 import org.apache.lucene.benchmark.byTask.tasks.CloseIndexTask;
 import org.apache.lucene.benchmark.byTask.tasks.CreateIndexTask;
@@ -44,42 +47,85 @@ public class LineDocSourceTest extends B
 
   private static final CompressorStreamFactory csFactory = new CompressorStreamFactory();
 
-  private void createBZ2LineFile(File file) throws Exception {
+  private void createBZ2LineFile(File file, boolean addHeader) throws Exception {
     OutputStream out = new FileOutputStream(file);
     out = csFactory.createCompressorOutputStream("bzip2", out);
     BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out, "utf-8"));
+    writeDocsToFile(writer, addHeader, null);
+    writer.close();
+  }
+
+  private void writeDocsToFile(BufferedWriter writer, boolean addHeader, Properties otherFields) throws IOException {
+    if (addHeader) {
+      writer.write(WriteLineDocTask.FIELDS_HEADER_INDICATOR);
+      writer.write(WriteLineDocTask.SEP);
+      writer.write(DocMaker.TITLE_FIELD);
+      writer.write(WriteLineDocTask.SEP);
+      writer.write(DocMaker.DATE_FIELD);
+      writer.write(WriteLineDocTask.SEP);
+      writer.write(DocMaker.BODY_FIELD);
+      if (otherFields!=null) {
+        // additional field names in the header 
+        for (Object fn : otherFields.keySet()) {
+          writer.write(WriteLineDocTask.SEP);
+          writer.write(fn.toString());
+        }
+      }
+      writer.newLine();
+    }
     StringBuilder doc = new StringBuilder();
-    doc.append("title").append(WriteLineDocTask.SEP).append("date").append(WriteLineDocTask.SEP).append("body");
+    doc.append("title").append(WriteLineDocTask.SEP).append("date").append(WriteLineDocTask.SEP).append(DocMaker.BODY_FIELD);
+    if (otherFields!=null) {
+      // additional field values in the doc line 
+      for (Object fv : otherFields.values()) {
+        doc.append(WriteLineDocTask.SEP).append(fv.toString());
+      }
+    }
     writer.write(doc.toString());
     writer.newLine();
+  }
+
+  private void createRegularLineFile(File file, boolean addHeader) throws Exception {
+    OutputStream out = new FileOutputStream(file);
+    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out, "utf-8"));
+    writeDocsToFile(writer, addHeader, null);
     writer.close();
   }
 
-  private void createRegularLineFile(File file) throws Exception {
+  private void createRegularLineFileWithMoreFields(File file, String...extraFields) throws Exception {
     OutputStream out = new FileOutputStream(file);
     BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out, "utf-8"));
-    StringBuilder doc = new StringBuilder();
-    doc.append("title").append(WriteLineDocTask.SEP).append("date").append(WriteLineDocTask.SEP).append("body");
-    writer.write(doc.toString());
-    writer.newLine();
+    Properties p = new Properties();
+    for (String f : extraFields) {
+      p.setProperty(f, f);
+    }
+    writeDocsToFile(writer, true, p);
     writer.close();
   }
   
-  private void doIndexAndSearchTest(File file, boolean setBZCompress,
-      String bz2CompressVal) throws Exception {
+  private void doIndexAndSearchTest(File file, Class<? extends LineParser> lineParserClass, String storedField) throws Exception {
+    doIndexAndSearchTestWithRepeats(file, lineParserClass, 1, storedField); // no extra repetitions
+    doIndexAndSearchTestWithRepeats(file, lineParserClass, 2, storedField); // 1 extra repetition
+    doIndexAndSearchTestWithRepeats(file, lineParserClass, 4, storedField); // 3 extra repetitions
+  }
+  
+  private void doIndexAndSearchTestWithRepeats(File file, 
+      Class<? extends LineParser> lineParserClass, int numAdds, String storedField) throws Exception {
 
     Properties props = new Properties();
     
     // LineDocSource specific settings.
     props.setProperty("docs.file", file.getAbsolutePath());
-    if (setBZCompress) {
-      props.setProperty("bzip.compression", bz2CompressVal);
+    if (lineParserClass != null) {
+      props.setProperty("line.parser", lineParserClass.getName());
     }
     
     // Indexing configuration.
     props.setProperty("analyzer", MockAnalyzer.class.getName());
     props.setProperty("content.source", LineDocSource.class.getName());
     props.setProperty("directory", "RAMDirectory");
+    props.setProperty("doc.stored", "true");
+    props.setProperty("doc.index.props", "true");
     
     // Create PerfRunData
     Config config = new Config(props);
@@ -87,34 +133,54 @@ public class LineDocSourceTest extends B
 
     TaskSequence tasks = new TaskSequence(runData, "testBzip2", null, false);
     tasks.addTask(new CreateIndexTask(runData));
-    tasks.addTask(new AddDocTask(runData));
+    for (int i=0; i<numAdds; i++) {
+      tasks.addTask(new AddDocTask(runData));
+    }
     tasks.addTask(new CloseIndexTask(runData));
     tasks.doLogic();
     
     IndexSearcher searcher = new IndexSearcher(runData.getDirectory(), true);
     TopDocs td = searcher.search(new TermQuery(new Term("body", "body")), 10);
-    assertEquals(1, td.totalHits);
+    assertEquals(numAdds, td.totalHits);
     assertNotNull(td.scoreDocs[0]);
+    
+    if (storedField==null) {
+      storedField = DocMaker.BODY_FIELD; // added to all docs and satisfies field-name == value
+    }
+    assertEquals("Wrong field value", storedField, searcher.doc(0).get(storedField));
+
     searcher.close();
   }
   
   /* Tests LineDocSource with a bzip2 input stream. */
   public void testBZip2() throws Exception {
     File file = new File(getWorkDir(), "one-line.bz2");
-    createBZ2LineFile(file);
-    doIndexAndSearchTest(file, true, "true");
+    createBZ2LineFile(file,true);
+    doIndexAndSearchTest(file, null, null);
   }
-  
-  public void testBZip2AutoDetect() throws Exception {
+
+  public void testBZip2NoHeaderLine() throws Exception {
     File file = new File(getWorkDir(), "one-line.bz2");
-    createBZ2LineFile(file);
-    doIndexAndSearchTest(file, false, null);
+    createBZ2LineFile(file,false);
+    doIndexAndSearchTest(file, null, null);
   }
   
   public void testRegularFile() throws Exception {
     File file = new File(getWorkDir(), "one-line");
-    createRegularLineFile(file);
-    doIndexAndSearchTest(file, false, null);
+    createRegularLineFile(file,true);
+    doIndexAndSearchTest(file, null, null);
+  }
+
+  public void testRegularFileSpecialHeader() throws Exception {
+    File file = new File(getWorkDir(), "one-line");
+    createRegularLineFile(file,true);
+    doIndexAndSearchTest(file, HeaderLineParser.class, null);
+  }
+
+  public void testRegularFileNoHeaderLine() throws Exception {
+    File file = new File(getWorkDir(), "one-line");
+    createRegularLineFile(file,false);
+    doIndexAndSearchTest(file, null, null);
   }
 
   public void testInvalidFormat() throws Exception {
@@ -134,7 +200,7 @@ public class LineDocSourceTest extends B
       writer.newLine();
       writer.close();
       try {
-        doIndexAndSearchTest(file, false, null);
+        doIndexAndSearchTest(file, null, null);
         fail("Some exception should have been thrown for: [" + testCases[i] + "]");
       } catch (Exception e) {
         // expected.
@@ -142,4 +208,19 @@ public class LineDocSourceTest extends B
     }
   }
   
+  /** Doc Name is not part of the default header */
+  public void testWithDocsName()  throws Exception {
+    File file = new File(getWorkDir(), "one-line");
+    createRegularLineFileWithMoreFields(file, DocMaker.NAME_FIELD);
+    doIndexAndSearchTest(file, null, DocMaker.NAME_FIELD);
+  }
+
+  /** Use fields names that are not defined in Docmaker and so will go to Properties */
+  public void testWithProperties()  throws Exception {
+    File file = new File(getWorkDir(), "one-line");
+    String specialField = "mySpecialField";
+    createRegularLineFileWithMoreFields(file, specialField);
+    doIndexAndSearchTest(file, null, specialField);
+  }
+  
 }

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java Mon Mar 28 10:50:28 2011
@@ -389,7 +389,7 @@ public class TrecContentSourceTest exten
       gotExpectedException = true;
     }
     assertTrue("Should have gotten NoMoreDataException!", gotExpectedException);
-    assertEquals("Wrong numbre of documents created by osurce!",5,n);
+    assertEquals("Wrong number of documents created by source!",5,n);
     assertTrue("Did not see all types!",unseenTypes.isEmpty());
   }
 

Modified: lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTaskTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTaskTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTaskTest.java (original)
+++ lucene/dev/branches/docvalues/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/tasks/WriteLineDocTaskTest.java Mon Mar 28 10:50:28 2011
@@ -31,6 +31,7 @@ import org.apache.lucene.benchmark.Bench
 import org.apache.lucene.benchmark.byTask.PerfRunData;
 import org.apache.lucene.benchmark.byTask.feeds.DocMaker;
 import org.apache.lucene.benchmark.byTask.utils.Config;
+import org.apache.lucene.benchmark.byTask.utils.StreamUtils.Type;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Index;
@@ -98,6 +99,25 @@ public class WriteLineDocTaskTest extend
       return doc;
     }
   }
+
+  // class has to be public so that Class.forName.newInstance() will work
+  // same as JustDate just that this one is treated as legal
+  public static final class LegalJustDateDocMaker extends DocMaker {
+    @Override
+    public Document makeDocument() throws Exception {
+      Document doc = new Document();
+      doc.add(new Field(DATE_FIELD, "date", Store.NO, Index.NOT_ANALYZED_NO_NORMS));
+      return doc;
+    }
+  }
+
+  // class has to be public so that Class.forName.newInstance() will work
+  public static final class EmptyDocMaker extends DocMaker {
+    @Override
+    public Document makeDocument() throws Exception {
+      return new Document();
+    }
+  }
   
   // class has to be public so that Class.forName.newInstance() will work
   public static final class ThreadingDocMaker extends DocMaker {
@@ -116,29 +136,43 @@ public class WriteLineDocTaskTest extend
 
   private static final CompressorStreamFactory csFactory = new CompressorStreamFactory();
 
-  private PerfRunData createPerfRunData(File file, boolean setBZCompress,
-                                        String bz2CompressVal,
+  private PerfRunData createPerfRunData(File file, 
+                                        boolean allowEmptyDocs,
                                         String docMakerName) throws Exception {
     Properties props = new Properties();
     props.setProperty("doc.maker", docMakerName);
     props.setProperty("line.file.out", file.getAbsolutePath());
-    if (setBZCompress) {
-      props.setProperty("bzip.compression", bz2CompressVal);
-    }
     props.setProperty("directory", "RAMDirectory"); // no accidental FS dir.
+    if (allowEmptyDocs) {
+      props.setProperty("sufficient.fields", ",");
+    }
+    if (docMakerName.equals(LegalJustDateDocMaker.class.getName())) {
+      props.setProperty("line.fields", DocMaker.DATE_FIELD);
+      props.setProperty("sufficient.fields", DocMaker.DATE_FIELD);
+    }
     Config config = new Config(props);
     return new PerfRunData(config);
   }
   
-  private void doReadTest(File file, boolean bz2File, String expTitle,
+  private void doReadTest(File file, Type fileType, String expTitle,
                           String expDate, String expBody) throws Exception {
     InputStream in = new FileInputStream(file);
-    if (bz2File) {
-      in = csFactory.createCompressorInputStream("bzip2", in);
+    switch(fileType) {
+    	case BZIP2:
+    		in = csFactory.createCompressorInputStream(CompressorStreamFactory.BZIP2, in);
+    		break;
+    	case GZIP:
+    		in = csFactory.createCompressorInputStream(CompressorStreamFactory.GZIP, in);
+    	case PLAIN:
+    		break; // nothing to do
+    	default:
+    		assertFalse("Unknown file type!",true); //fail, should not happen
     }
     BufferedReader br = new BufferedReader(new InputStreamReader(in, "utf-8"));
     try {
       String line = br.readLine();
+      assertHeaderLine(line);
+      line = br.readLine();
       assertNotNull(line);
       String[] parts = line.split(Character.toString(WriteLineDocTask.SEP));
       int numExpParts = expBody == null ? 2 : 3;
@@ -153,42 +187,47 @@ public class WriteLineDocTaskTest extend
       br.close();
     }
   }
+
+  private void assertHeaderLine(String line) {
+    assertTrue("First line should be a header line",line.startsWith(WriteLineDocTask.FIELDS_HEADER_INDICATOR));
+  }
   
   /* Tests WriteLineDocTask with a bzip2 format. */
   public void testBZip2() throws Exception {
     
     // Create a document in bz2 format.
     File file = new File(getWorkDir(), "one-line.bz2");
-    PerfRunData runData = createPerfRunData(file, true, "true", WriteLineDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, WriteLineDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
     
-    doReadTest(file, true, "title", "date", "body");
+    doReadTest(file, Type.BZIP2, "title", "date", "body");
   }
   
-  public void testBZip2AutoDetect() throws Exception {
+  /* Tests WriteLineDocTask with a gzip format. */
+  public void testGZip() throws Exception {
     
-    // Create a document in bz2 format.
-    File file = new File(getWorkDir(), "one-line.bz2");
-    PerfRunData runData = createPerfRunData(file, false, null, WriteLineDocMaker.class.getName());
+    // Create a document in gz format.
+    File file = new File(getWorkDir(), "one-line.gz");
+    PerfRunData runData = createPerfRunData(file, false, WriteLineDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
     
-    doReadTest(file, true, "title", "date", "body");
+    doReadTest(file, Type.GZIP, "title", "date", "body");
   }
   
   public void testRegularFile() throws Exception {
     
     // Create a document in regular format.
     File file = new File(getWorkDir(), "one-line");
-    PerfRunData runData = createPerfRunData(file, true, "false", WriteLineDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, WriteLineDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
     
-    doReadTest(file, false, "title", "date", "body");
+    doReadTest(file, Type.PLAIN, "title", "date", "body");
   }
 
   public void testCharsReplace() throws Exception {
@@ -196,12 +235,12 @@ public class WriteLineDocTaskTest extend
     // separator char. However, it didn't replace newline characters, which
     // resulted in errors in LineDocSource.
     File file = new File(getWorkDir(), "one-line");
-    PerfRunData runData = createPerfRunData(file, false, null, NewLinesDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, NewLinesDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
     
-    doReadTest(file, false, "title text", "date text", "body text two");
+    doReadTest(file, Type.PLAIN, "title text", "date text", "body text two");
   }
   
   public void testEmptyBody() throws Exception {
@@ -209,27 +248,28 @@ public class WriteLineDocTaskTest extend
     // had a TITLE element (LUCENE-1755). It should throw away documents if they
     // don't have BODY nor TITLE
     File file = new File(getWorkDir(), "one-line");
-    PerfRunData runData = createPerfRunData(file, false, null, NoBodyDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, NoBodyDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
     
-    doReadTest(file, false, "title", "date", null);
+    doReadTest(file, Type.PLAIN, "title", "date", null);
   }
   
   public void testEmptyTitle() throws Exception {
     File file = new File(getWorkDir(), "one-line");
-    PerfRunData runData = createPerfRunData(file, false, null, NoTitleDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, NoTitleDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
     
-    doReadTest(file, false, "", "date", "body");
+    doReadTest(file, Type.PLAIN, "", "date", "body");
   }
   
+  /** Fail by default when there's only date */
   public void testJustDate() throws Exception {
     File file = new File(getWorkDir(), "one-line");
-    PerfRunData runData = createPerfRunData(file, false, null, JustDateDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, JustDateDocMaker.class.getName());
     WriteLineDocTask wldt = new WriteLineDocTask(runData);
     wldt.doLogic();
     wldt.close();
@@ -237,15 +277,53 @@ public class WriteLineDocTaskTest extend
     BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(file), "utf-8"));
     try {
       String line = br.readLine();
+      assertHeaderLine(line);
+      line = br.readLine();
       assertNull(line);
     } finally {
       br.close();
     }
   }
 
+  public void testLegalJustDate() throws Exception {
+    File file = new File(getWorkDir(), "one-line");
+    PerfRunData runData = createPerfRunData(file, false, LegalJustDateDocMaker.class.getName());
+    WriteLineDocTask wldt = new WriteLineDocTask(runData);
+    wldt.doLogic();
+    wldt.close();
+    
+    BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(file), "utf-8"));
+    try {
+      String line = br.readLine();
+      assertHeaderLine(line);
+      line = br.readLine();
+      assertNotNull(line);
+    } finally {
+      br.close();
+    }
+  }
+
+  public void testEmptyDoc() throws Exception {
+    File file = new File(getWorkDir(), "one-line");
+    PerfRunData runData = createPerfRunData(file, true, EmptyDocMaker.class.getName());
+    WriteLineDocTask wldt = new WriteLineDocTask(runData);
+    wldt.doLogic();
+    wldt.close();
+    
+    BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(file), "utf-8"));
+    try {
+      String line = br.readLine();
+      assertHeaderLine(line);
+      line = br.readLine();
+      assertNotNull(line);
+    } finally {
+      br.close();
+    }
+  }
+
   public void testMultiThreaded() throws Exception {
     File file = new File(getWorkDir(), "one-line");
-    PerfRunData runData = createPerfRunData(file, false, null, ThreadingDocMaker.class.getName());
+    PerfRunData runData = createPerfRunData(file, false, ThreadingDocMaker.class.getName());
     final WriteLineDocTask wldt = new WriteLineDocTask(runData);
     Thread[] threads = new Thread[10];
     for (int i = 0; i < threads.length; i++) {
@@ -269,8 +347,10 @@ public class WriteLineDocTaskTest extend
     Set<String> ids = new HashSet<String>();
     BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(file), "utf-8"));
     try {
+      String line = br.readLine();
+      assertHeaderLine(line); // header line is written once, no matter how many threads there are
       for (int i = 0; i < threads.length; i++) {
-        String line = br.readLine();
+        line = br.readLine();
         String[] parts = line.split(Character.toString(WriteLineDocTask.SEP));
         assertEquals(3, parts.length);
         // check that all thread names written are the same in the same line

Modified: lucene/dev/branches/docvalues/modules/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/modules/build.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/modules/build.xml (original)
+++ lucene/dev/branches/docvalues/modules/build.xml Mon Mar 28 10:50:28 2011
@@ -18,6 +18,7 @@
  -->
 
 <project name="modules" default="test" basedir=".">
+  <import file="../common-build.xml"/>
   <target name="test" description="Test all modules">
     <sequential>
       <subant target="test" inheritall="false" failonerror="true">
@@ -27,7 +28,7 @@
     </sequential>
   </target>
 
-  <target name="compile" description="Compile all modules">
+  <target name="compile" description="Compile all modules" depends="validate-modules">
     <sequential>
       <subant target="compile" inheritall="false" failonerror="true">
         <fileset dir="analysis" includes="build.xml" />
@@ -63,6 +64,25 @@
       </subant>
     </sequential>
   </target>
+  <target name="validate" depends="validate-modules"/>
+  <target name="validate-modules" depends="check-legal-modules" unless="validated-modules"/>
+  <target name="check-legal-modules" depends="compile-tools">
+    <java classname="org.apache.lucene.validation.DependencyChecker" failonerror="true" fork="true">
+      <classpath>
+        <path refid="tools.runtime.classpath" />
+      </classpath>
+      <!-- TODO: it might be better to just automatically find all directories that contain jar files, but that could take a
+       long time.  This should be faster, but we could miss a directory
+       -->
+      <!-- Modules -->
+      <arg value="-c" />
+      <arg value="${basedir}/analysis/icu/lib" />
+      <arg value="-c" />
+      <arg value="${basedir}/analysis/phonetic/lib" />
+      <arg value="-c" />
+      <arg value="${basedir}/benchmark/lib" />
+    </java>
+  </target>
 
   <target name="clean" description="Clean all modules">
     <sequential>

Modified: lucene/dev/branches/docvalues/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/CHANGES.txt?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/CHANGES.txt (original)
+++ lucene/dev/branches/docvalues/solr/CHANGES.txt Mon Mar 28 10:50:28 2011
@@ -25,12 +25,13 @@ $Id$
 Versions of Major Components
 ---------------------
 Apache Lucene trunk
-Apache Tika 0.8-SNAPSHOT
+Apache Tika 0.8
 Carrot2 3.4.2
 Velocity 1.6.4 and Velocity Tools 2.0
+Apache UIMA 2.3.1-SNAPSHOT
 
 
-Upgrading from Solr 3.1-dev
+Upgrading from Solr 3.2-dev
 ----------------------
 
 * The Lucene index format has changed and as a result, once you upgrade, 
@@ -107,6 +108,13 @@ New Features
   Adding a parameter NOW=<time_in_ms> to the request will override the
   current time.  (Peter Sturge, yonik)
 
+* SOLR-1566: Transforming documents in the ResponseWriters.  This will allow
+  for more complex results in responses and open the door for function queries
+  as results. (ryan with patches from grant, noble, cmale, yonik)
+
+* SOLR-2417: Add explain info directly to return documents using ?fl=_explain_ (ryan)
+  
+
 Optimizations
 ----------------------
 
@@ -153,6 +161,11 @@ Bug Fixes
 * SOLR-2275: fix DisMax 'mm' parsing to be tolerant of whitespace
   (Erick Erickson via hossman)
 
+* SOLR-309: Fix FieldType so setting an analyzer on a FieldType that
+  doesn't expect it will generate an error.  Practically speaking this
+  means that Solr will now correctly generate an error on
+  initialization if the schema.xml contains an analyzer configuration
+  for a fieldType that does not use TextField.  (hossman) 
 
 Other Changes
 ----------------------
@@ -175,18 +188,39 @@ Other Changes
   using Generics where applicable in method/object declatations, and
   adding @SuppressWarnings("unchecked") when appropriate (hossman)
 
+* SOLR-2375: Suggester Lookup implementations now store trie data
+  and load it back on init. This means that large tries don't have to be
+  rebuilt on every commit or core reload. (ab)
+
+* SOLR-2413: Support for returning multi-valued fields w/o <arr> tag 
+  in the XMLResponseWriter was removed.  XMLResponseWriter only 
+  no longer work with values less then 2.2 (ryan)
+
+* SOLR-2423: FieldType argument changed from String to Object
+  Conversion from SolrInputDocument > Object > Fieldable is now managed
+  by FieldType rather then DocumentBuilder.  (ryan)
+
+
 Documentation
 ----------------------
 
 * SOLR-2232: Improved README info on solr.solr.home in examples
   (Eric Pugh and hossman)
 
+==================  3.2.0-dev ==================
+Versions of Major Components
+---------------------
+Apache Lucene trunk
+Apache Tika 0.8
+Carrot2 3.4.2
+
+(No Changes)
 
 ==================  3.1.0-dev ==================
 Versions of Major Components
 ---------------------
 Apache Lucene trunk
-Apache Tika 0.8-SNAPSHOT
+Apache Tika 0.8
 Carrot2 3.4.2
 
 
@@ -309,8 +343,6 @@ New Features
 * SOLR-1379: Add RAMDirectoryFactory for non-persistent in memory index storage.
   (Alex Baranov via yonik)
 
-* SOLR-1769: Solr 1.4 Replication - Repeater throwing NullPointerException (Jörgen Rydenius via noble)  
-
 * SOLR-1857: Synced Solr analysis with Lucene 3.1. Added KeywordMarkerFilterFactory 
   and StemmerOverrideFilterFactory, which can be used to tune stemming algorithms. 
   Added factories for Bulgarian, Czech, Hindi, Turkish, and Wikipedia analysis. Improved the
@@ -375,7 +407,7 @@ New Features
 * SOLR-1240: "Range Faceting" has been added.  This is a generalization
   of the existing "Date Faceting" logic so that it now supports any
   all stock numeric field types that support range queries in addition
-  to dates.  
+  to dates. facet.date is now deprecated in favor of this generalized mechanism.
   (Gijs Kunze, hossman)
 
 * SOLR-2021: Add SolrEncoder plugin to Highlighter. (koji)
@@ -404,7 +436,7 @@ New Features
   Example: q=add($v1,$v2)&v1=mul(popularity,5)&v2=20.0
   (yonik)
 
-* SOLR-2133: Function query parser can now parse multiple coma separated
+* SOLR-2133: Function query parser can now parse multiple comma separated
   value sources.  It also now fails if there is extra unexpected text
   after parsing the functions, instead of silently ignoring it.
   This allows expressions like q=dist(2,vector(1,2),$pt)&pt=3,4   (yonik)
@@ -452,6 +484,8 @@ Optimizations
 
 Bug Fixes
 ----------------------
+* SOLR-1769: Solr 1.4 Replication - Repeater throwing NullPointerException (Jörgen Rydenius via noble)  
+
 * SOLR-1432: Make the new ValueSource.getValues(context,reader) delegate
   to the original ValueSource.getValues(reader) so custom sources
   will work. (yonik)
@@ -618,23 +652,11 @@ Bug Fixes
 
 * SOLR-2148: Highlighter doesn't support q.alt. (koji)
 
-* SOLR-1794: Dataimport of CLOB fields fails when getCharacterStream() is 
-  defined in a superclass. (Gunnar Gauslaa Bergem via rmuir)
-
 * SOLR-2180: It was possible for EmbeddedSolrServer to leave searchers
   open if a request threw an exception. (yonik)
 
 * SOLR-2173: Suggester should always rebuild Lookup data if Lookup.load fails. (ab)
 
-* SOLR-2057: DataImportHandler never calls UpdateRequestProcessor.finish()
-  (Drew Farris via koji)
-
-* SOLR-1973: Empty fields in XML update messages confuse DataImportHandler. (koji)
-
-* SOLR-2221: Use StrUtils.parseBool() to get values of boolean options in DIH.
-  true/on/yes (for TRUE) and false/off/no (for FALSE) can be used for sub-options
-  (debug, verbose, synchronous, commit, clean, optimize) for full/delta-import commands. (koji)
-
 * SOLR-2081: BaseResponseWriter.isStreamingDocs causes
   SingleResponseWriter.end to be called 2x 
   (Chris A. Mattmann via hossman) 
@@ -680,7 +702,7 @@ Bug Fixes
 
 * SOLR-96: Fix XML parsing in XMLUpdateRequestHandler and
   DocumentAnalysisRequestHandler to respect charset from XML file and only
-  use HTTP header's "Content-Type" as a "hint". (Uwe Schindler)
+  use HTTP header's "Content-Type" as a "hint". (uschindler)
 
 * SOLR-2339: Fix sorting to explicitly generate an error if you
   attempt to sort on a multiValued field. (hossman)
@@ -688,10 +710,22 @@ Bug Fixes
 * SOLR-2348: Fix field types to explicitly generate an error if you
   attempt to get a ValueSource for a multiValued field. (hossman)
 
-* SOLR-1191: resolve DataImportHandler deltaQuery column against pk when pk
-  has a prefix (e.g. pk="book.id" deltaQuery="select id from ..."). More
-  useful error reporting when no match found (previously failed with a
-  NullPointerException in log and no clear user feedback). (gthb via yonik)
+* SOLR-2380: Distributed faceting could miss values when facet.sort=index
+  and when facet.offset was greater than 0. (yonik)
+  
+* SOLR-1656: XIncludes and other HREFs in XML files loaded by ResourceLoader
+  are fixed to be resolved using the URI standard (RFC 2396). The system 
+  identifier is no longer a plain filename with path, it gets initialized 
+  using a custom URI scheme "solrres:". This scheme is resolved using a 
+  EntityResolver that utilizes ResourceLoader 
+  (org.apache.solr.common.util.SystemIdResolver). This makes all relative 
+  pathes in Solr's config files behave like expected. This change 
+  introduces some backwards breaks in the API: Some config classes 
+  (Config, SolrConfig, IndexSchema) were changed to take 
+  org.xml.sax.InputSource instead of InputStream. There may also be some 
+  backwards breaks in existing config files, it is recommended to check 
+  your config files / XSLTs and replace all XIncludes/HREFs that were 
+  hacked to use absolute paths to use relative ones. (uschindler)
 
 
 Other Changes
@@ -798,6 +832,25 @@ Other Changes
 
 * SOLR-2365: Move DIH jars out of solr.war (David Smiley via yonik)
 
+* SOLR-2381: Include a patched version of Jetty (6.1.26 + JETTY-1340) 
+  to fix problematic UTF-8 handling for supplementary characters.
+  (Bernd Fehling, uschindler, yonik, rmuir)
+
+* SOLR-2391: The preferred Content-Type for XML was changed to
+  application/xml. XMLResponseWriter now only delivers using this
+  type; updating documents and analyzing documents is still supported
+  using text/xml as Content-Type, too. If you have clients that are
+  hardcoded on text/xml as Content-Type, you have to change them.
+  (uschindler, rmuir)
+
+* SOLR-2414: All ResponseWriters now use only ServletOutputStreams
+  and wrap their own Writer around it when serializing. This fixes
+  the bug in PHPSerializedResponseWriter that produced wrong string
+  length if the servlet container had a broken UTF-8 encoding that was
+  in fact CESU-8 (see SOLR-1091). The system property to enable the
+  CESU-8 byte counting in PHPSerializesResponseWriters for broken
+  servlet containers was therefore removed and is now ignored if set.
+  Output is always UTF-8.  (uschindler, yonik, rmuir)
 
 Build
 ----------------------

Modified: lucene/dev/branches/docvalues/solr/LICENSE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/LICENSE.txt?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/LICENSE.txt (original)
+++ lucene/dev/branches/docvalues/solr/LICENSE.txt Mon Mar 28 10:50:28 2011
@@ -261,9 +261,9 @@ such code.
 1.13. You (or Your) means an individual or a legal entity exercising rights
 under, and complying with all of the terms of, this License. For legal
 entities, You includes any entity which controls, is controlled by, or is under
-common control with You. For purposes of this definition, control means (a)�the
+common control with You. For purposes of this definition, control means (a) the
 power, direct or indirect, to cause the direction or management of such entity,
-whether by contract or otherwise, or (b)�ownership of more than fifty percent
+whether by contract or otherwise, or (b) ownership of more than fifty percent
 (50%) of the outstanding shares or beneficial ownership of such entity.
 
 2. License Grants.
@@ -278,12 +278,12 @@ with or without Modifications, and/or as
 Patent Claims infringed by the making, using or selling of Original Software,
 to make, have made, use, practice, sell, and offer for sale, and/or otherwise
 dispose of the Original Software (or portions thereof).  (c) The licenses
-granted in Sections�2.1(a) and (b) are effective on the date Initial Developer
+granted in Sections 2.1(a) and (b) are effective on the date Initial Developer
 first distributes or otherwise makes the Original Software available to a third
-party under the terms of this License.  (d) Notwithstanding Section�2.1(b)
-above, no patent license is granted: (1)�for code that You delete from the
-Original Software, or (2)�for infringements caused by: (i)�the modification of
-the Original Software, or (ii)�the combination of the Original Software with
+party under the terms of this License.  (d) Notwithstanding Section 2.1(b)
+above, no patent license is granted: (1) for code that You delete from the
+Original Software, or (2) for infringements caused by: (i) the modification of
+the Original Software, or (ii) the combination of the Original Software with
 other software or devices.
 
 2.2. Contributor Grant.  Conditioned upon Your compliance with Section 3.1
@@ -297,17 +297,17 @@ and/or as part of a Larger Work; and (b)
 making, using, or selling of Modifications made by that Contributor either
 alone and/or in combination with its Contributor Version (or portions of such
 combination), to make, use, sell, offer for sale, have made, and/or otherwise
-dispose of: (1)�Modifications made by that Contributor (or portions thereof);
-and (2)�the combination of Modifications made by that Contributor with its
+dispose of: (1) Modifications made by that Contributor (or portions thereof);
+and (2) the combination of Modifications made by that Contributor with its
 Contributor Version (or portions of such combination).  (c) The licenses
-granted in Sections�2.2(a) and 2.2(b) are effective on the date Contributor
+granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor
 first distributes or otherwise makes the Modifications available to a third
-party.  (d) Notwithstanding Section�2.2(b) above, no patent license is granted:
-(1)�for any code that Contributor has deleted from the Contributor Version;
-(2)�for infringements caused by: (i)�third party modifications of Contributor
-Version, or (ii)�the combination of Modifications made by that Contributor with
+party.  (d) Notwithstanding Section 2.2(b) above, no patent license is granted:
+(1) for any code that Contributor has deleted from the Contributor Version;
+(2) for infringements caused by: (i) third party modifications of Contributor
+Version, or (ii) the combination of Modifications made by that Contributor with
 other software (except as part of the Contributor Version) or other devices; or
-(3)�under Patent Claims infringed by Covered Software in the absence of
+(3) under Patent Claims infringed by Covered Software in the absence of
 Modifications made by that Contributor.
 
 3. Distribution Obligations.
@@ -389,9 +389,9 @@ License published by the license steward
 
 When You are an Initial Developer and You want to create a new license for Your
 Original Software, You may create and use a modified version of this License if
-You: (a)�rename the license and remove any references to the name of the
+You: (a) rename the license and remove any references to the name of the
 license steward (except to note that the license differs from this License);
-and (b)�otherwise make it clear that the license contains terms which differ
+and (b) otherwise make it clear that the license contains terms which differ
 from this License.
 
 5. DISCLAIMER OF WARRANTY.
@@ -422,14 +422,14 @@ the Participant is a Contributor or the 
 is the Initial Developer) directly or indirectly infringes any patent, then any
 and all rights granted directly or indirectly to You by such Participant, the
 Initial Developer (if the Initial Developer is not the Participant) and all
-Contributors under Sections�2.1 and/or 2.2 of this License shall, upon 60 days
+Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days
 notice from Participant terminate prospectively and automatically at the
 expiration of such 60 day notice period, unless if within such 60 day period
 You withdraw Your claim with respect to the Participant Software against such
 Participant either unilaterally or pursuant to a written agreement with
 Participant.
 
-6.3. In the event of termination under Sections�6.1 or 6.2 above, all end user
+6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user
 licenses that have been validly granted by You or any distributor hereunder
 prior to termination (excluding licenses granted to You by any distributor)
 shall survive termination.
@@ -453,9 +453,9 @@ LIMITATION MAY NOT APPLY TO YOU.
 8. U.S. GOVERNMENT END USERS.
 
 The Covered Software is a commercial item, as that term is defined in
-48�C.F.R.�2.101 (Oct. 1995), consisting of commercial computer software (as
-that term is defined at 48 C.F.R. �252.227-7014(a)(1)) and commercial computer
-software documentation as such terms are used in 48�C.F.R.�12.212 (Sept. 1995).
+48 C.F.R. 2.101 (Oct. 1995), consisting of commercial computer software (as
+that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and commercial computer
+software documentation as such terms are used in 48 C.F.R. 12.212 Sept. 1995).
 Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4
 (June 1995), all U.S. Government End Users acquire Covered Software with only
 those rights set forth herein. This U.S. Government Rights clause is in lieu
@@ -534,7 +534,7 @@ ANY THEORY OF LIABILITY, WHETHER IN CONT
 SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
 ==========================================================================
-The following license applies to easymock.jar
+The following license applies to easymock-2.2.jar
 --------------------------------------------------------------------------
 EasyMock 2 License (MIT License)
 Copyright (c) 2001-2007 OFFIS, Tammo Freese.

Modified: lucene/dev/branches/docvalues/solr/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/NOTICE.txt?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/NOTICE.txt (original)
+++ lucene/dev/branches/docvalues/solr/NOTICE.txt Mon Mar 28 10:50:28 2011
@@ -1,6 +1,6 @@
 ==============================================================
  Apache Solr
- Copyright 2006-2010 The Apache Software Foundation
+ Copyright 2006-2011 The Apache Software Foundation
 ==============================================================
 
 This product includes software developed by
@@ -38,15 +38,114 @@ License: Common Development and Distribu
 This product includes the HSQL Database (HSQLDB) 1.8.0.10 jar: http://hsqldb.org/
 License: http://hsqldb.org/web/hsqlLicense.html
 
+This product includes code (JaspellTernarySearchTrie) from Java Spelling Checking Package (jaspell): http://jaspell.sourceforge.net/
+License: The BSD License (http://www.opensource.org/licenses/bsd-license.php)
 
 =========================================================================
 ==  Apache Lucene Notice                                               ==
 =========================================================================
+Includes lib/servlet-api-2.4.jar from  Apache Tomcat
+Includes lib/ant-1.7.1.jar and lib/ant-junit-1.7.1.jar from Apache Ant
+Includes contrib/queries/lib/jakarta-regexp-1.4.jar from Apache Jakarta Regexp
+
+ICU4J, (under contrib/icu) is licensed under an MIT styles license
+(contrib/icu/lib/ICU-LICENSE.txt) and Copyright (c) 1995-2008 
+International Business Machines Corporation and others
+
+Some data files (under contrib/icu/src/data) are derived from Unicode data such
+as the Unicode Character Database. See http://unicode.org/copyright.html for more
+details.
+
+Brics Automaton (under src/java/org/apache/lucene/util/automaton) is 
+BSD-licensed, created by Anders Møller. See http://www.brics.dk/automaton/
+
+The levenshtein automata tables (under src/java/org/apache/lucene/util/automaton) were
+automatically generated with the moman/finenight FSA library, created by
+Jean-Philippe Barrette-LaPierre. This library is available under an MIT license,
+see http://sites.google.com/site/rrettesite/moman and 
+http://bitbucket.org/jpbarrette/moman/overview/
+
+The class org.apache.lucene.SorterTemplate was inspired by CGLIB's class with
+the same name. The implementation part is mainly done using pre-existing
+Lucene sorting code. In-place stable mergesort was borrowed from CGLIB,
+which is Apache-licensed.
+
+The Google Code Prettify is Apache License 2.0.
+See http://code.google.com/p/google-code-prettify/
+
+JUnit (under lib/junit-4.7.jar) is licensed under the Common Public License v. 1.0
+See http://junit.sourceforge.net/cpl-v10.html
+
+JLine (under contrib/lucli/lib/jline.jar) is licensed under the BSD License.
+See http://jline.sourceforge.net/
+
+=========================================================================
+==  Apache Lucene Benchmark Notice                                     ==
+=========================================================================
+Includes software from other Apache Software Foundation projects,
+including, but not limited to:
+ - Commons Beanutils (lib/commons-beanutils-1.7.0.jar)
+ - Commons Collections (lib/commons-collections-3.1.jar)
+ - Commons Compress (lib/commons-compress-1.0.jar)
+ - Commons Digester (lib/commons-digester-1.7.jar)
+ - Commons Logging (lib/commons-logging-1.0.4.jar)
+ - Xerces (lib/xercesImpl-2.9.1-patched-XERCESJ-1257.jar)
+
+=========================================================================
+==  Apache Lucene Analyzers Notice                                     ==
+========================================================================= 
+Includes software from other Apache Software Foundation projects,
+including, but not limited to:
+  - Apache Commons
+
 The snowball stemmers in
-  contrib/snowball/src/java/net/sf/snowball
+  common/src/java/net/sf/snowball
+were developed by Martin Porter and Richard Boulton.
+The snowball stopword lists in
+  common/src/resources/org/apache/lucene/analysis/snowball
 were developed by Martin Porter and Richard Boulton.
 The full snowball package is available from
   http://snowball.tartarus.org/
+
+The Arabic,Persian,Romanian,Bulgarian, and Hindi analyzers (common) come with a default
+stopword list that is BSD-licensed created by Jacques Savoy.  These files reside in:
+common/src/resources/org/apache/lucene/analysis/ar/stopwords.txt,
+common/src/resources/org/apache/lucene/analysis/fa/stopwords.txt,
+common/src/resources/org/apache/lucene/analysis/ro/stopwords.txt,
+common/src/resources/org/apache/lucene/analysis/bg/stopwords.txt,
+common/src/resources/org/apache/lucene/analysis/hi/stopwords.txt
+See http://members.unine.ch/jacques.savoy/clef/index.html.
+
+The German,Spanish,Finnish,French,Hungarian,Italian,Portuguese,Russian and Swedish light stemmers
+(common) are based on BSD-licensed reference implementations created by Jacques Savoy and
+Ljiljana Dolamic. These files reside in:
+common/src/java/org/apache/lucene/analysis/de/GermanLightStemmer.java
+common/src/java/org/apache/lucene/analysis/de/GermanMinimalStemmer.java
+common/src/java/org/apache/lucene/analysis/es/SpanishLightStemmer.java
+common/src/java/org/apache/lucene/analysis/fi/FinnishLightStemmer.java
+common/src/java/org/apache/lucene/analysis/fr/FrenchLightStemmer.java
+common/src/java/org/apache/lucene/analysis/fr/FrenchMinimalStemmer.java
+common/src/java/org/apache/lucene/analysis/hu/HungarianLightStemmer.java
+common/src/java/org/apache/lucene/analysis/it/ItalianLightStemmer.java
+common/src/java/org/apache/lucene/analysis/pt/PortugueseLightStemmer.java
+common/src/java/org/apache/lucene/analysis/ru/RussianLightStemmer.java
+common/src/java/org/apache/lucene/analysis/sv/SwedishLightStemmer.java
+
+The Stempel analyzer (stempel) includes BSD-licensed software developed 
+by the Egothor project http://egothor.sf.net/, created by Leo Galambos, Martin Kvapil,
+and Edmond Nolan.
+
+The Polish analyzer (stempel) comes with a default
+stopword list that is BSD-licensed created by the Carrot2 project. The file resides
+in stempel/src/resources/org/apache/lucene/analysis/pl/stopwords.txt.
+See http://project.carrot2.org/license.html.
+
+The SmartChineseAnalyzer source code (smartcn) was
+provided by Xiaoping Gao and copyright 2009 by www.imdict.net.
+
+WordBreakTestUnicode_*.java (under modules/analysis/common/src/test/) 
+is derived from Unicode data such as the Unicode Character Database. 
+See http://unicode.org/copyright.html for more details.
 ---
 
 This product includes/uses software, Woodstox (http://woodstox.codehaus.org),
@@ -143,6 +242,12 @@ Copyright (c) 2003-2005, www.fontbox.org
 Copyright (c) 1995-2005 International Business Machines Corporation and others
 
 Copyright (c) 2000-2005 INRIA, France Telecom
+
+Copyright 2001-2005 (C) MetaStuff, Ltd. All Rights Reserved.
+
+Copyright 2004 Sun Microsystems, Inc. (Rome JAR)
+
+Copyright 2002-2008 by John Cowan (TagSoup -- http://ccil.org/~cowan/XML/tagsoup/)
  
 
 =========================================================================
@@ -160,7 +265,7 @@ See http://project.carrot2.org/
 ==     Guava Notice                                                    ==
 =========================================================================
 
-Copyright ???? Google, Inc.
+Copyright (C) 2009 Google Inc.
 
 This product includes software developed by the Google Guava project.
 
@@ -170,7 +275,7 @@ See http://code.google.com/p/guava-libra
 ==     Prettify Notice                                                 ==
 =========================================================================
 
-Copyright ???? Google, Inc.
+Copyright (C) 2009 Google Inc.
 
 This product includes software developed by the Google Prettify project.
 
@@ -179,7 +284,7 @@ See http://code.google.com/p/google-code
 =========================================================================
 ==     Jackson Notice                                                  ==
 =========================================================================
-Copyright ????
+Copyright 2010 FasterXML, LLC
 
 This product includes software developed by the Jackson project.
 

Modified: lucene/dev/branches/docvalues/solr/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/README.txt?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/README.txt (original)
+++ lucene/dev/branches/docvalues/solr/README.txt Mon Mar 28 10:50:28 2011
@@ -17,8 +17,8 @@
 Welcome to the Apache Solr project!
 -----------------------------------
 
-Apache Solr is a search server based on the Apache Lucene search
-library. 
+Solr is the popular, blazing fast open source enterprise search platform
+from the Apache Lucene project.
 
 For a complete description of the Solr project, team composition, source
 code repositories, and other details, please see the Solr web site at
@@ -29,12 +29,20 @@ Getting Started
 ---------------
 
 See the "example" directory for an example Solr setup.  A tutorial
-using the example setup can be found in "docs/tutorial.html" 
+using the example setup can be found at
+http://lucene.apache.org/solr/tutorial.html
+or in in "docs/tutorial.html" in a binary distribution.
 
 
 
-Files Included In Apache Solr Distributions
--------------------------------------------
+Files included in an Apache Solr binary distribution
+----------------------------------------------------
+
+example/
+  A self-contained example Solr instance, complete with a sample
+  configuration, documents to index, and the Jetty Servlet container.
+  Please see example/README.txt for information about running this
+  example.
 
 dist/apache-solr-XX.war
   The Apache Solr Application.  Deploy this WAR file to any servlet
@@ -45,21 +53,12 @@ dist/apache-solr-XX.jar
   Apache Solr Plugins (see http://wiki.apache.org/solr/SolrPlugins for
   more information).
 
-example/
-  A self-contained example Solr instance, complete with a sample
-  configuration, documents to index, and the Jetty Servlet container.
-  Please see example/README.txt for information about running this
-  example.
-
 docs/index.html
   The contents of the Apache Solr website.
   
 docs/api/index.html
   The Apache Solr Javadoc API documentation.
 
-src/
-  The Apache Solr source code.
-
 
 
 Instructions for Building Apache Solr from Source
@@ -82,7 +81,7 @@ Instructions for Building Apache Solr fr
 
      http://lucene.apache.org/solr/version_control.html
 
-4. Navigate to that folder and issue an "ant" command to see the available options
+4. Navigate to the "solr" folder and issue an "ant" command to see the available options
    for building, testing, and packaging Solr.
   
    NOTE: