You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/02/22 02:01:11 UTC

svn commit: r1073192 [22/32] - in /lucene/dev/branches/realtime_search: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/demo/ dev-tools/idea/lucene/contrib/highlighter/ dev-tools/idea/lucene/...

Added: lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecLATimesParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecLATimesParser.java?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecLATimesParser.java (added)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecLATimesParser.java Tue Feb 22 01:00:39 2011
@@ -0,0 +1,71 @@
+package org.apache.lucene.benchmark.byTask.feeds;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Date;
+
+/**
+ * Parser for the FT docs in trec disks 4+5 collection format
+ */
+public class TrecLATimesParser extends TrecDocParser {
+
+  private static final String DATE = "<DATE>";
+  private static final String DATE_END = "</DATE>";
+  private static final String DATE_NOISE = "day,"; // anything aftre the ',' 
+
+  private static final String SUBJECT = "<SUBJECT>";
+  private static final String SUBJECT_END = "</SUBJECT>";
+  private static final String HEADLINE = "<HEADLINE>";
+  private static final String HEADLINE_END = "</HEADLINE>";
+  
+  @Override
+  public DocData parse(DocData docData, String name, TrecContentSource trecSrc, 
+      StringBuilder docBuf, ParsePathType pathType) throws IOException, InterruptedException {
+    int mark = 0; // that much is skipped
+
+    // date...
+    Date date = null;
+    String dateStr = extract(docBuf, DATE, DATE_END, -1, null);
+    if (dateStr != null) {
+      int d2a = dateStr.indexOf(DATE_NOISE);
+      if (d2a > 0) {
+        dateStr = dateStr.substring(0,d2a+3); // we need the "day" part
+      }
+      dateStr = stripTags(dateStr,0).toString();
+      date = trecSrc.parseDate(dateStr.trim());
+    }
+     
+    // title... first try with SUBJECT, them with HEADLINE
+    String title = extract(docBuf, SUBJECT, SUBJECT_END, -1, null);
+    if (title==null) {
+      title = extract(docBuf, HEADLINE, HEADLINE_END, -1, null);
+    }
+    if (title!=null) {
+      title = stripTags(title,0).toString().trim();
+    }
+    
+    docData.clear();
+    docData.setName(name);
+    docData.setDate(date);
+    docData.setTitle(title);
+    docData.setBody(stripTags(docBuf, mark).toString());
+    return docData;
+  }
+
+}

Added: lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecParserByPath.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecParserByPath.java?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecParserByPath.java (added)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/TrecParserByPath.java Tue Feb 22 01:00:39 2011
@@ -0,0 +1,33 @@
+package org.apache.lucene.benchmark.byTask.feeds;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+/**
+ * Parser for trec docs which selects the parser to apply according 
+ * to the source files path, defaulting to {@link TrecGov2Parser}.
+ */
+public class TrecParserByPath extends TrecDocParser {
+
+  @Override
+  public DocData parse(DocData docData, String name, TrecContentSource trecSrc, 
+      StringBuilder docBuf, ParsePathType pathType) throws IOException, InterruptedException {
+    return pathType2parser.get(pathType).parse(docData, name, trecSrc, docBuf, pathType);
+  }
+
+}

Modified: lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/NearRealtimeReaderTask.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/NearRealtimeReaderTask.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/NearRealtimeReaderTask.java (original)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/NearRealtimeReaderTask.java Tue Feb 22 01:00:39 2011
@@ -59,7 +59,7 @@ public class NearRealtimeReaderTask exte
     }
     
     long t = System.currentTimeMillis();
-    IndexReader r = IndexReader.open(w);
+    IndexReader r = IndexReader.open(w, true);
     runData.setIndexReader(r);
     // Transfer our reference to runData
     r.decRef();

Modified: lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/StringBuilderReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/StringBuilderReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/StringBuilderReader.java (original)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/StringBuilderReader.java Tue Feb 22 01:00:39 2011
@@ -158,8 +158,10 @@ public class StringBuilderReader extends
     synchronized (lock) {
       this.sb = sb;
       length = sb.length();
+      next = mark = 0;
     }
   }
+  
   @Override
   public long skip(long ns) throws IOException {
     synchronized (lock) {

Modified: lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/SubmissionReport.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/SubmissionReport.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/SubmissionReport.java (original)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/quality/utils/SubmissionReport.java Tue Feb 22 01:00:39 2011
@@ -28,7 +28,7 @@ import org.apache.lucene.search.TopDocs;
 /**
  * Create a log ready for submission.
  * Extend this class and override
- * {@link #report(QualityQuery, TopDocs, String, Searcher)}
+ * {@link #report(QualityQuery, TopDocs, String, IndexSearcher)}
  * to create different reports. 
  */
 public class SubmissionReport {

Modified: lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java (original)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java Tue Feb 22 01:00:39 2011
@@ -29,146 +29,119 @@ import java.util.regex.Pattern;
 /**
  * Split the Reuters SGML documents into Simple Text files containing: Title, Date, Dateline, Body
  */
-public class ExtractReuters
-{
-    private File reutersDir;
-    private File outputDir;
-    private static final String LINE_SEPARATOR = System.getProperty("line.separator");
-
-    public ExtractReuters(File reutersDir, File outputDir)
-    {
-        this.reutersDir = reutersDir;
-        this.outputDir = outputDir;
-        System.out.println("Deleting all files in " + outputDir);
-        File [] files = outputDir.listFiles();
-        for (int i = 0; i < files.length; i++)
-        {
-            files[i].delete();
-        }
-
+public class ExtractReuters {
+  private File reutersDir;
+  private File outputDir;
+  private static final String LINE_SEPARATOR = System.getProperty("line.separator");
+
+  public ExtractReuters(File reutersDir, File outputDir) {
+    this.reutersDir = reutersDir;
+    this.outputDir = outputDir;
+    System.out.println("Deleting all files in " + outputDir);
+    for (File f : outputDir.listFiles()) {
+      f.delete();
     }
+  }
 
-    public void extract()
-    {
-        File [] sgmFiles = reutersDir.listFiles(new FileFilter()
-        {
-            public boolean accept(File file)
-            {
-                return file.getName().endsWith(".sgm");
-            }
-        });
-        if (sgmFiles != null && sgmFiles.length > 0)
-        {
-            for (int i = 0; i < sgmFiles.length; i++)
-            {
-                File sgmFile = sgmFiles[i];
-                extractFile(sgmFile);
-            }
-        }
-        else
-        {
-            System.err.println("No .sgm files in " + reutersDir);
-        }
+  public void extract() {
+    File[] sgmFiles = reutersDir.listFiles(new FileFilter() {
+      public boolean accept(File file) {
+        return file.getName().endsWith(".sgm");
+      }
+    });
+    if (sgmFiles != null && sgmFiles.length > 0) {
+      for (File sgmFile : sgmFiles) {
+        extractFile(sgmFile);
+      }
+    } else {
+      System.err.println("No .sgm files in " + reutersDir);
     }
+  }
 
-    Pattern EXTRACTION_PATTERN = Pattern.compile("<TITLE>(.*?)</TITLE>|<DATE>(.*?)</DATE>|<BODY>(.*?)</BODY>");
-
-    private static String[] META_CHARS
-            = {"&", "<", ">", "\"", "'"};
+  Pattern EXTRACTION_PATTERN = Pattern
+      .compile("<TITLE>(.*?)</TITLE>|<DATE>(.*?)</DATE>|<BODY>(.*?)</BODY>");
 
-    private static String[] META_CHARS_SERIALIZATIONS
-            = {"&amp;", "&lt;", "&gt;", "&quot;", "&apos;"};
+  private static String[] META_CHARS = { "&", "<", ">", "\"", "'" };
 
-    /**
-     * Override if you wish to change what is extracted
-     *
-     * @param sgmFile
-     */
-    protected void extractFile(File sgmFile)
-    {
-        try
-        {
-            BufferedReader reader = new BufferedReader(new FileReader(sgmFile));
-
-            StringBuilder buffer = new StringBuilder(1024);
-            StringBuilder outBuffer = new StringBuilder(1024);
-
-            String line = null;
-            int docNumber = 0;
-            while ((line = reader.readLine()) != null)
-            {
-                //when we see a closing reuters tag, flush the file
-
-                if (line.indexOf("</REUTERS") == -1) {
-                    //Replace the SGM escape sequences
-
-                    buffer.append(line).append(' ');//accumulate the strings for now, then apply regular expression to get the pieces,
-                }
-                else
-                {
-                    //Extract the relevant pieces and write to a file in the output dir
-                    Matcher matcher = EXTRACTION_PATTERN.matcher(buffer);
-                    while (matcher.find())
-                    {
-                        for (int i = 1; i <= matcher.groupCount(); i++)
-                        {
-                            if (matcher.group(i) != null)
-                            {
-                                outBuffer.append(matcher.group(i));
-                            }
-                        }
-                        outBuffer.append(LINE_SEPARATOR).append(LINE_SEPARATOR);
-                    }
-                    String out = outBuffer.toString();
-                    for (int i = 0; i < META_CHARS_SERIALIZATIONS.length; i++)
-                    {
-                        out = out.replaceAll(META_CHARS_SERIALIZATIONS[i], META_CHARS[i]);
-                    }
-                    File outFile = new File(outputDir, sgmFile.getName() + "-" + (docNumber++) + ".txt");
-                    //System.out.println("Writing " + outFile);
-                    FileWriter writer = new FileWriter(outFile);
-                    writer.write(out);
-                    writer.close();
-                    outBuffer.setLength(0);
-                    buffer.setLength(0);
-                }
+  private static String[] META_CHARS_SERIALIZATIONS = { "&amp;", "&lt;",
+      "&gt;", "&quot;", "&apos;" };
+
+  /**
+   * Override if you wish to change what is extracted
+   * 
+   * @param sgmFile
+   */
+  protected void extractFile(File sgmFile) {
+    try {
+      BufferedReader reader = new BufferedReader(new FileReader(sgmFile));
+
+      StringBuilder buffer = new StringBuilder(1024);
+      StringBuilder outBuffer = new StringBuilder(1024);
+
+      String line = null;
+      int docNumber = 0;
+      while ((line = reader.readLine()) != null) {
+        // when we see a closing reuters tag, flush the file
+
+        if (line.indexOf("</REUTERS") == -1) {
+          // Replace the SGM escape sequences
+
+          buffer.append(line).append(' ');// accumulate the strings for now,
+                                          // then apply regular expression to
+                                          // get the pieces,
+        } else {
+          // Extract the relevant pieces and write to a file in the output dir
+          Matcher matcher = EXTRACTION_PATTERN.matcher(buffer);
+          while (matcher.find()) {
+            for (int i = 1; i <= matcher.groupCount(); i++) {
+              if (matcher.group(i) != null) {
+                outBuffer.append(matcher.group(i));
+              }
             }
-            reader.close();
-        }
-
-        catch (
-                IOException e
-                )
-
-        {
-            throw new RuntimeException(e);
-        }
+            outBuffer.append(LINE_SEPARATOR).append(LINE_SEPARATOR);
+          }
+          String out = outBuffer.toString();
+          for (int i = 0; i < META_CHARS_SERIALIZATIONS.length; i++) {
+            out = out.replaceAll(META_CHARS_SERIALIZATIONS[i], META_CHARS[i]);
+          }
+          File outFile = new File(outputDir, sgmFile.getName() + "-"
+              + (docNumber++) + ".txt");
+          // System.out.println("Writing " + outFile);
+          FileWriter writer = new FileWriter(outFile);
+          writer.write(out);
+          writer.close();
+          outBuffer.setLength(0);
+          buffer.setLength(0);
+        }
+      }
+      reader.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
     }
+  }
 
-
-    public static void main(String[] args)
-    {
-        if (args.length != 2)
-        {
-            printUsage();
-        }
-        File reutersDir = new File(args[0]);
-
-        if (reutersDir.exists())
-        {
-            File outputDir = new File(args[1]);
-            outputDir.mkdirs();
-            ExtractReuters extractor = new ExtractReuters(reutersDir, outputDir);
-            extractor.extract();
-        }
-        else
-        {
-            printUsage();
-        }
+  public static void main(String[] args) {
+    if (args.length != 2) {
+      printUsage();
     }
-
-    private static void printUsage()
-    {
-        System.err.println("Usage: java -cp <...> org.apache.lucene.benchmark.utils.ExtractReuters <Path to Reuters SGM files> <Output Path>");
+    File reutersDir = new File(args[0]);
+    if (!reutersDir.exists()) {
+      printUsage();
+      return;
     }
+    
+    // First, extract to a tmp directory and only if everything succeeds, rename
+    // to output directory.
+    File outputDir = new File(args[1] + "-tmp");
+    outputDir.mkdirs();
+    ExtractReuters extractor = new ExtractReuters(reutersDir, outputDir);
+    extractor.extract();
+    // Now rename to requested output dir
+    outputDir.renameTo(new File(args[1]));
+  }
+
+  private static void printUsage() {
+    System.err.println("Usage: java -cp <...> org.apache.lucene.benchmark.utils.ExtractReuters <Path to Reuters SGM files> <Output Path>");
+  }
+  
 }

Modified: lucene/dev/branches/realtime_search/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java (original)
+++ lucene/dev/branches/realtime_search/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/TrecContentSourceTest.java Tue Feb 22 01:00:39 2011
@@ -18,14 +18,20 @@ package org.apache.lucene.benchmark.byTa
  */
 
 import java.io.BufferedReader;
+import java.io.File;
 import java.io.IOException;
 import java.io.StringReader;
 import java.text.ParseException;
+import java.util.Arrays;
 import java.util.Date;
+import java.util.HashSet;
+import java.util.Properties;
 
+import org.apache.lucene.benchmark.byTask.feeds.TrecDocParser.ParsePathType;
 import org.apache.lucene.benchmark.byTask.utils.Config;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
 
 public class TrecContentSourceTest extends LuceneTestCase {
 
@@ -329,5 +335,62 @@ public class TrecContentSourceTest exten
     // Don't test that NoMoreDataException is thrown, since the forever flag is
     // turned on.
   }
+  
+  /** 
+   * Open a trec content source over a directory with files of all trec path types and all
+   * supported formats - bzip, gzip, txt. 
+   */
+  public void testTrecFeedDirAllTypes() throws Exception {
+    File dataDir =  _TestUtil.getTempDir("trecFeedAllTypes");
+    _TestUtil.unzip(getDataFile("trecdocs.zip"), dataDir);
+    TrecContentSource tcs = new TrecContentSource();
+    Properties props = new Properties();
+    props.setProperty("print.props", "false");
+    props.setProperty("content.source.verbose", "false");
+    props.setProperty("content.source.excludeIteration", "true");
+    props.setProperty("doc.maker.forever", "false");
+    props.setProperty("docs.dir", dataDir.getCanonicalPath().replace('\\','/')); 
+    props.setProperty("trec.doc.parser", TrecParserByPath.class.getName());
+    props.setProperty("content.source.forever", "false");
+    tcs.setConfig(new Config(props));
+    tcs.resetInputs();
+    DocData dd = new DocData();
+    int n = 0;
+    boolean gotExpectedException = false;
+    HashSet<ParsePathType> unseenTypes = new HashSet<ParsePathType>(Arrays.asList(ParsePathType.values()));
+    try {
+      while (n<100) { // arbiterary limit to prevent looping forever in case of test failure
+        dd = tcs.getNextDocData(dd);
+        ++n;
+        assertNotNull("doc data "+n+" should not be null!", dd);
+        unseenTypes.remove(tcs.currPathType);
+        switch(tcs.currPathType) {
+          case GOV2:
+            assertDocData(dd, "TEST-000", "TEST-000 title", "TEST-000 text", tcs.parseDate("Sun, 11 Jan 2009 08:00:00 GMT"));
+            break;
+          case FBIS:
+            assertDocData(dd, "TEST-001", "TEST-001 Title", "TEST-001 text", tcs.parseDate("1 January 1991"));
+            break;
+          case FR94:
+            // no title extraction in this source for now
+            assertDocData(dd, "TEST-002", null, "DEPARTMENT OF SOMETHING", tcs.parseDate("February 3, 1994"));
+            break;
+          case FT:
+            assertDocData(dd, "TEST-003", "Test-003 title", "Some pub text", tcs.parseDate("980424"));
+            break;
+          case LATIMES:
+            assertDocData(dd, "TEST-004", "Test-004 Title", "Some paragraph", tcs.parseDate("January 17, 1997, Sunday"));
+            break;
+          default:
+            assertTrue("Should never get here!", false);
+        }
+      }
+    } catch (NoMoreDataException e) {
+      gotExpectedException = true;
+    }
+    assertTrue("Should have gotten NoMoreDataException!", gotExpectedException);
+    assertEquals("Wrong numbre of documents created by osurce!",5,n);
+    assertTrue("Did not see all types!",unseenTypes.isEmpty());
+  }
 
 }

Added: lucene/dev/branches/realtime_search/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/trecdocs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/benchmark/src/test/org/apache/lucene/benchmark/byTask/feeds/trecdocs.zip?rev=1073192&view=auto
==============================================================================
Binary file - no diff available.

Modified: lucene/dev/branches/realtime_search/modules/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/modules/build.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/modules/build.xml (original)
+++ lucene/dev/branches/realtime_search/modules/build.xml Tue Feb 22 01:00:39 2011
@@ -56,7 +56,7 @@
 
   <target name="generate-maven-artifacts" description="Generate Maven Artifacts for Modules">
     <sequential>
-      <ant target="get-maven-poms" inheritAll="false" dir=".."/>
+      <ant target="get-maven-poms" dir=".."/>
       <subant target="dist-maven" inheritall="false" failonerror="true">
         <fileset dir="analysis" includes="build.xml" />
         <fileset dir="benchmark" includes="build.xml" />

Modified: lucene/dev/branches/realtime_search/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/CHANGES.txt?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/CHANGES.txt (original)
+++ lucene/dev/branches/realtime_search/solr/CHANGES.txt Tue Feb 22 01:00:39 2011
@@ -53,6 +53,7 @@ Upgrading from Solr 3.1-dev
   legacy behavior should set a default value for the 'mm' param in
   their solrconfig.xml file.
 
+
 Detailed Change List
 ----------------------
 
@@ -106,8 +107,6 @@ New Features
   Adding a parameter NOW=<time_in_ms> to the request will override the
   current time.  (Peter Sturge, yonik)
 
-* SOLR-2325: Allow tagging and exlcusion of main query for faceting. (yonik)
-
 Optimizations
 ----------------------
 
@@ -154,12 +153,6 @@ Bug Fixes
 * SOLR-2275: fix DisMax 'mm' parsing to be tolerant of whitespace
   (Erick Erickson via hossman)
 
-* SOLR-2307: fix bug in PHPSerializedResponseWriter (wt=phps) when
-  dealing with SolrDocumentList objects -- ie: sharded queries.
-  (Antonio Verni via hossman)
-  
-* SOLR-2127: Fixed serialization of default core and indentation of solr.xml when serializing.
-  (Ephraim Ofir, Mark Miller)
 
 Other Changes
 ----------------------
@@ -188,6 +181,7 @@ Documentation
 * SOLR-2232: Improved README info on solr.solr.home in examples
   (Eric Pugh and hossman)
 
+
 ==================  3.1.0-dev ==================
 Versions of Major Components
 ---------------------
@@ -239,6 +233,19 @@ Upgrading from Solr 1.4
 
 * readercycle script was removed. (SOLR-2046)
 
+* In previous releases, sorting or evaluating function queries on 
+  fields that were "multiValued" (either by explicit declaration in
+  schema.xml or by implict behavior because the "version" attribute on
+  the schema was less then 1.2) did not generally work, but it would
+  sometimes silently act as if it succeeded and order the docs
+  arbitrarily.  Solr will now fail on any attempt to sort, or apply a
+  function to, multi-valued fields 
+
+* The DataImportHandler jars are no longer included in the solr
+  WAR and should be added in Solr's lib directory, or referenced
+  via the <lib> directive in solrconfig.xml.
+
+
 Detailed Change List
 ----------------------
 
@@ -254,7 +261,7 @@ New Features
 * SOLR-1553: New dismax parser implementation (accessible as "edismax")
   that supports full lucene syntax, improved reserved char escaping,
   fielded queries, improved proximity boosting, and improved stopword
-  handling. (yonik)
+  handling. Note: status is experimental for now. (yonik)
 
 * SOLR-1574: Add many new functions from java Math (e.g. sin, cos) (yonik)
 
@@ -393,7 +400,6 @@ New Features
   	     http://wiki.apache.org/solr/SpatialSearch and the example.  Refactored some items in Lucene spatial. 
 	     Removed SpatialTileField as the underlying CartesianTier is broken beyond repair and is going to be moved. (gsingers)
 
-
 * SOLR-2128: Full parameter substitution for function queries.
   Example: q=add($v1,$v2)&v1=mul(popularity,5)&v2=20.0
   (yonik)
@@ -414,6 +420,15 @@ New Features
 * SOLR-2129: Added a Solr module for dynamic metadata extraction/indexing with Apache UIMA.
   See contrib/uima/README.txt for more information.  (Tommaso Teofili via rmuir)
 
+* SOLR-2325: Allow tagging and exlcusion of main query for faceting. (yonik)
+
+* SOLR-2263: Add ability for RawResponseWriter to stream binary files as well as
+  text files.  (Eric Pugh via yonik)
+
+* SOLR-860: Add debug output for MoreLikeThis. (koji)
+
+* SOLR-1057: Add PathHierarchyTokenizerFactory. (ryan, koji)
+
 Optimizations
 ----------------------
 
@@ -511,7 +526,8 @@ Bug Fixes
 * SOLR-1711: SolrJ - StreamingUpdateSolrServer had a race condition that
   could halt the streaming of documents. The original patch to fix this
   (never officially released) introduced another hanging bug due to
-  connections not being released.  (Attila Babo, Erik Hetzner via yonik)
+  connections not being released.
+  (Attila Babo, Erik Hetzner, Johannes Tuchscherer via yonik)
   
 * SOLR-1748, SOLR-1747, SOLR-1746, SOLR-1745, SOLR-1744: Streams and Readers
   retrieved from ContentStreams are not closed in various places, resulting
@@ -635,6 +651,48 @@ Bug Fixes
 * SOLR-2261: fix velocity template layout.vm that referred to an older
   version of jquery.  (Eric Pugh via rmuir)
 
+* SOLR-2307: fix bug in PHPSerializedResponseWriter (wt=phps) when
+  dealing with SolrDocumentList objects -- ie: sharded queries.
+  (Antonio Verni via hossman)
+
+* SOLR-2127: Fixed serialization of default core and indentation of solr.xml when serializing.
+  (Ephraim Ofir, Mark Miller)
+
+* SOLR-2320: Fixed ReplicationHandler detail reporting for masters
+  (hossman)
+
+* SOLR-482: Provide more exception handling in CSVLoader (gsingers)
+
+* SOLR-1283: HTMLStripCharFilter sometimes threw a "Mark Invalid" exception. 
+  (Julien Coloos, hossman, yonik)
+
+* SOLR-2085: Improve SolrJ behavior when FacetComponent comes before
+  QueryComponent (Tomas Salfischberger via hossman)
+
+* SOLR-1940: Fix SolrDispatchFilter behavior when Content-Type is
+  unknown (Lance Norskog and hossman)
+
+* SOLR-1983: snappuller fails when modifiedConfFiles is not empty and
+  full copy of index is needed. (Alexander Kanarsky via yonik)
+
+* SOLR-2156: SnapPuller fails to clean Old Index Directories on Full Copy
+  (Jayendra Patil via yonik)
+
+* 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)
+
+* SOLR-2339: Fix sorting to explicitly generate an error if you
+  attempt to sort on a multiValued field. (hossman)
+
+* 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)
+
 
 Other Changes
 ----------------------
@@ -728,6 +786,18 @@ Other Changes
   
 * SOLR-2213: Upgrade to jQuery 1.4.3 (Erick Erickson via ryan)
 
+* SOLR-1826: Add unit tests for highlighting with termOffsets=true
+  and overlapping tokens. (Stefan Oestreicher via rmuir)
+
+* SOLR-2340: Add version infos to message in JavaBinCodec when throwing
+  exception. (koji)
+
+* SOLR-2350: Since Solr no longer requires XML files to be in UTF-8
+  (see SOLR-96) SimplePostTool (aka: post.jar) has been improved to
+  work with files of any mime-type or charset. (hossman)
+
+* SOLR-2365: Move DIH jars out of solr.war (David Smiley via yonik)
+
 
 Build
 ----------------------
@@ -744,7 +814,6 @@ Build
 * LUCENE-2657: Switch from using Maven POM templates to full POMs when
   generating Maven artifacts (Steven Rowe)
 
-
 Documentation
 ----------------------
 
@@ -862,8 +931,7 @@ New Features
  7. SOLR-680: Add StatsComponent. This gets simple statistics on matched numeric fields,
     including: min, max, mean, median, stddev.  (koji, ryan)
 
-    7.1 SOLR-1380: Added support for multi-valued fields to stats component
-        (Harish Agarwal via gsingers)
+    7.1 SOLR-1380: Added support for multi-valued fields (Harish Agarwal via gsingers)
 
  8. SOLR-561: Added Replication implemented in Java as a request handler. Supports index replication
     as well as configuration replication and exposes detailed statistics and progress information

Modified: lucene/dev/branches/realtime_search/solr/NOTICE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/NOTICE.txt?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/NOTICE.txt (original)
+++ lucene/dev/branches/realtime_search/solr/NOTICE.txt Tue Feb 22 01:00:39 2011
@@ -156,7 +156,6 @@ This product includes software developed
 
 See http://project.carrot2.org/
 
-
 =========================================================================
 ==     Guava Notice                                                    ==
 =========================================================================
@@ -168,6 +167,16 @@ This product includes software developed
 See http://code.google.com/p/guava-libraries/
 
 =========================================================================
+==     Prettify Notice                                                 ==
+=========================================================================
+
+Copyright ???? Google, Inc.
+
+This product includes software developed by the Google Prettify project.
+
+See http://code.google.com/p/google-code-prettify/
+
+=========================================================================
 ==     Jackson Notice                                                  ==
 =========================================================================
 Copyright ????

Modified: lucene/dev/branches/realtime_search/solr/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/build.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/build.xml (original)
+++ lucene/dev/branches/realtime_search/solr/build.xml Tue Feb 22 01:00:39 2011
@@ -18,6 +18,8 @@
 -->
 
 <project name="solr" default="usage" basedir="." xmlns:artifact="antlib:org.apache.maven.artifact.ant">
+
+  <property name="solr-path" value="." />
 	
   <import file="common-build.xml"/>
   
@@ -340,7 +342,7 @@
     <path refid="compile.classpath.solrj" />
     <pathelement location="${dest}/solr"/>
     <pathelement location="${dest}/solrj"/> <!-- include solrj -->
-    <pathelement location="${common-solr.dir}/../lucene/build/classes/test" />  <!-- include some lucene test code -->
+    <pathelement location="${common-solr.dir}/../lucene/build/classes/test-framework" />  <!-- include some lucene test code -->
   </path>
 
   <path id="test.run.classpath">
@@ -348,7 +350,7 @@
     <pathelement location="${dest}/tests"/>
     <!-- include the solrj classpath and jetty files included in example -->
     <path refid="compile.classpath.solrj" />
-    <pathelement location="${common-solr.dir}/../lucene/build/classes/test" />  <!-- include some lucene test code -->
+    <pathelement location="${common-solr.dir}/../lucene/build/classes/test-framework" />  <!-- include some lucene test code -->
     <pathelement path="${java.class.path}"/>
   </path>
 
@@ -754,6 +756,12 @@
         excludes="*.tgz *.zip *.md5 **/*src*.jar **/*docs*.jar" />
       <tarfileset dir="${build.docs}"
         prefix="${fullnamever}/docs/" />
+      <tarfileset dir="../lucene"
+        prefix="lucene"
+        excludes="**/build/" />
+      <tarfileset dir="../modules"
+        prefix="modules"
+        excludes="**/build/" />
     </tar>
     <solr-checksum file="${dist}/${fullnamever}.tgz"/>
 
@@ -773,6 +781,12 @@
         prefix="${fullnamever}"
         includes="**/*.sh **/bin/ src/scripts/"
         filemode="755" />
+      <zipfileset dir="../lucene"
+        prefix="lucene"
+        excludes="**/build/" />
+      <zipfileset dir="../modules"
+        prefix="modules"
+        excludes="**/build/" />
     </zip>
     <solr-checksum file="${dist}/${fullnamever}.zip"/>
 
@@ -846,7 +860,7 @@
 
   <target name="generate-maven-artifacts" depends="maven.ant.tasks-check, create-package">
     <sequential>
-	  <ant target="get-maven-poms" inheritAll="false" dir=".."/>
+	  <ant target="get-maven-poms" dir=".."/>
 
       <!--
 
@@ -912,7 +926,7 @@
         </artifact-attachments>
       </m2-deploy>
 
-      <m2-deploy pom.xml="contrib/extraction/solr-uima-pom.xml.template"
+      <m2-deploy pom.xml="contrib/uima/pom.xml"
                  jar.file="${dist}/apache-solr-uima-${version}.jar">
         <artifact-attachments>
           <attach file="${dist}/apache-solr-uima-src-${version}.jar" classifier="sources"/>
@@ -956,6 +970,8 @@
     description="runs the tasks over src/java excluding the license directory">
     <rat:report xmlns:rat="antlib:org.apache.rat.anttasks">
       <fileset dir="src/java"/>
+      <fileset dir="src/test"/>
+      <fileset dir="src/webapp"/>
       <fileset dir="src/common"/>
       <fileset dir="src/solrj"/>
       <fileset dir="client">
@@ -963,6 +979,8 @@
       </fileset>
       <fileset dir="contrib/dataimporthandler/src/main/java"/>
       <fileset dir="contrib/dataimporthandler/src/test/java"/>
+      <fileset dir="contrib/dataimporthandler/src/extras/main/java"/>
+      <fileset dir="contrib/dataimporthandler/src/extras/test/java"/>
       <fileset dir="contrib/clustering/src/main/java"/>
       <fileset dir="contrib/clustering/src/test/java"/>
       <fileset dir="contrib/extraction/src/main/java"/>

Modified: lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/solr/conf/solrconfig.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/solr/conf/solrconfig.xml (original)
+++ lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/solr/conf/solrconfig.xml Tue Feb 22 01:00:39 2011
@@ -17,19 +17,11 @@
 -->
 
 <config>
-  <!-- Set this to 'false' if you want solr to continue working after it has 
-       encountered an severe configuration error.  In a production environment, 
-       you may want solr to keep working even if one handler is mis-configured.
-
-       You may also set this to false using by setting the system property:
-         -Dsolr.abortOnConfigurationError=false
-     -->
-  <abortOnConfigurationError>${solr.abortOnConfigurationError:true}</abortOnConfigurationError>
 
   <!-- Used to specify an alternate directory to hold all index data
        other than the default ./data under the Solr home.
        If replication is in use, this should match the replication configuration. -->
-  <dataDir>${solr.data.dir:./solr/data}</dataDir>
+  <dataDir>${solr.data.dir:}</dataDir>
 
   <indexDefaults>
    <!-- Values here affect all index writers and act as a default unless overridden. -->

Modified: lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/test/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/test/conf/solrconfig.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/test/conf/solrconfig.xml (original)
+++ lucene/dev/branches/realtime_search/solr/client/ruby/solr-ruby/test/conf/solrconfig.xml Tue Feb 22 01:00:39 2011
@@ -17,20 +17,12 @@
 -->
 
 <config>
-  <!-- Set this to 'false' if you want solr to continue working after it has 
-       encountered an severe configuration error.  In a production environment, 
-       you may want solr to keep working even if one handler is mis-configured.
-
-       You may also set this to false using by setting the system property:
-         -Dsolr.abortOnConfigurationError=false
-     -->
-  <abortOnConfigurationError>${solr.abortOnConfigurationError:true}</abortOnConfigurationError>
 
   <!-- Used to specify an alternate directory to hold all index data
        other than the default ./data under the Solr home.
        If replication is in use, this should match the replication configuration. -->
   <!--
-  <dataDir>./solr/data</dataDir>
+  <dataDir>${solr.data.dir:}</dataDir>
   -->
 
   <indexDefaults>

Modified: lucene/dev/branches/realtime_search/solr/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/common-build.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/common-build.xml (original)
+++ lucene/dev/branches/realtime_search/solr/common-build.xml Tue Feb 22 01:00:39 2011
@@ -33,6 +33,9 @@
 
   <property name="name" value="${ant.project.name}"/>
 
+  <property name="dev-tools.dir" value="${solr-path}/../dev-tools"/>
+  <property name="prettify.dir" value="${dev-tools.dir}/prettify"/>
+
   <tstamp>
     <format property="year" pattern="yyyy"/>
     <format property="DSTAMP" pattern="yyyy-MM-dd"/>
@@ -344,6 +347,10 @@
     <attribute name="destdir"/>
   	<attribute name="title" default="${Name} ${version} API (${specversion})"/>
     <sequential>
+      <mkdir dir="@{destdir}"/>
+      <copy todir="@{destdir}/prettify" overwrite="false">
+        <fileset dir="${prettify.dir}"/>
+      </copy>
       <javadoc
           packagenames="org.apache.solr.*"
           failonerror="true"
@@ -357,6 +364,7 @@
           link="${javadoc.link.java}"
           windowtitle="${Name} ${version} API"
           doctitle="@{title}"
+          stylesheetfile="@{destdir}/prettify/stylesheet+prettify.css"
           bottom="Copyright &amp;copy; ${year} Apache Software Foundation.  All Rights Reserved.">
         <tag name="todo" description="To Do:"/>
         <tag name="uml.property" description="UML Property:"/>
@@ -368,6 +376,10 @@
         <link href="${javadoc.link.java}"/>
         <link href="${javadoc.link.junit}"/>
         <link href="${javadoc.link.lucene}"/>
+      	<header><![CDATA[
+      		 <script src="{@docRoot}/prettify/prettify.js" type="text/javascript"></script>
+      		 <script language="JavaScript">window.onload=function(){windowTitle();prettyPrint();}</script>
+      	]]></header>
 
         <sources />
 

Modified: lucene/dev/branches/realtime_search/solr/contrib/analysis-extras/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/analysis-extras/build.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/analysis-extras/build.xml (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/analysis-extras/build.xml Tue Feb 22 01:00:39 2011
@@ -73,7 +73,7 @@
     <pathelement path="${dest}/test-classes"/>
     <pathelement path="${java.class.path}"/>
     <pathelement location="${common-solr.dir}/build/tests"/> <!-- include solr test code -->
-    <pathelement location="${common-solr.dir}/../lucene/build/classes/test" />  <!-- include some lucene test code -->
+    <pathelement location="${common-solr.dir}/../lucene/build/classes/test-framework" />  <!-- include some lucene test code -->
     <path refid="common.classpath"/>
   </path>
 

Modified: lucene/dev/branches/realtime_search/solr/contrib/clustering/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/clustering/build.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/clustering/build.xml (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/clustering/build.xml Tue Feb 22 01:00:39 2011
@@ -42,7 +42,7 @@
     <pathelement path="${dest}/test-classes"/>
     <pathelement path="${java.class.path}"/>
     <pathelement location="${common-solr.dir}/build/tests"/> <!-- include solr test code -->
-    <pathelement location="${common-solr.dir}/../lucene/build/classes/test" />  <!-- include some lucene test code -->
+    <pathelement location="${common-solr.dir}/../lucene/build/classes/test-framework" />  <!-- include some lucene test code -->
     <path refid="common.classpath"/>
     <!-- DistributedClusteringComponentTest uses Jetty -->
     <fileset dir="${solr-path}/example/lib">

Modified: lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/ClusteringComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/ClusteringComponent.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/ClusteringComponent.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/ClusteringComponent.java Tue Feb 22 01:00:39 2011
@@ -46,6 +46,13 @@ import java.util.Set;
  * <p/>
  * This engine is experimental.  Output from this engine is subject to change in future releases.
  *
+ * <pre class="prettyprint" >
+ * &lt;searchComponent class="org.apache.solr.handler.clustering.ClusteringComponent" name="clustering"&gt;
+ *   &lt;lst name="engine"&gt;
+ *     &lt;str name="name"&gt;default&lt;/str&gt;
+ *     &lt;str name="carrot.algorithm"&gt;org.carrot2.clustering.lingo.LingoClusteringAlgorithm&lt;/str&gt;
+ *   &lt;/lst&gt;
+ * &lt;/searchComponent&gt;</pre>
  */
 public class ClusteringComponent extends SearchComponent implements SolrCoreAware {
   private transient static Logger log = LoggerFactory.getLogger(ClusteringComponent.class);

Modified: lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/CarrotClusteringEngine.java Tue Feb 22 01:00:39 2011
@@ -77,6 +77,7 @@ public class CarrotClusteringEngine exte
 
   private String idFieldName;
 
+  @Override
   @Deprecated
   public Object cluster(Query query, DocList docList, SolrQueryRequest sreq) {
     SolrIndexSearcher searcher = sreq.getSearcher();
@@ -90,6 +91,7 @@ public class CarrotClusteringEngine exte
     }
   }
 
+  @Override
   public Object cluster(Query query, SolrDocumentList solrDocList,
       Map<SolrDocument, Integer> docIds, SolrQueryRequest sreq) {
     try {

Modified: lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/LuceneLanguageModelFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/LuceneLanguageModelFactory.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/LuceneLanguageModelFactory.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/clustering/src/main/java/org/apache/solr/handler/clustering/carrot2/LuceneLanguageModelFactory.java Tue Feb 22 01:00:39 2011
@@ -69,7 +69,8 @@ public class LuceneLanguageModelFactory 
 	/**
 	 * Provide an {@link IStemmer} implementation for a given language.
 	 */
-	protected IStemmer createStemmer(LanguageCode language) {
+	@Override
+  protected IStemmer createStemmer(LanguageCode language) {
 		switch (language) {
 		case ARABIC:
 			return ArabicStemmerFactory.createStemmer();

Modified: lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/MockDocumentClusteringEngine.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/MockDocumentClusteringEngine.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/MockDocumentClusteringEngine.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/java/org/apache/solr/handler/clustering/MockDocumentClusteringEngine.java Tue Feb 22 01:00:39 2011
@@ -25,11 +25,13 @@ import org.apache.solr.search.DocSet;
  *
  **/
 public class MockDocumentClusteringEngine extends DocumentClusteringEngine {
+  @Override
   public NamedList cluster(DocSet docs, SolrParams solrParams) {
     NamedList result = new NamedList();
     return result;
   }
 
+  @Override
   public NamedList cluster(SolrParams solrParams) {
     NamedList result = new NamedList();
     return result;

Modified: lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/clustering/src/test/resources/solr-clustering/conf/solrconfig.xml Tue Feb 22 01:00:39 2011
@@ -18,19 +18,11 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
-  <!-- Set this to 'false' if you want solr to continue working after it has 
-       encountered an severe configuration error.  In a production environment, 
-       you may want solr to keep working even if one handler is mis-configured.
-
-       You may also set this to false using by setting the system property:
-         -Dsolr.abortOnConfigurationError=false
-     -->
-  <abortOnConfigurationError>${solr.abortOnConfigurationError:true}</abortOnConfigurationError>
 
   <!-- Used to specify an alternate directory to hold all index data
        other than the default ./data under the Solr home.
        If replication is in use, this should match the replication configuration. -->
-  <dataDir>${solr.data.dir:./solr/data}</dataDir>
+  <dataDir>${solr.data.dir:}</dataDir>
 
 
   <indexDefaults>

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/CHANGES.txt?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/CHANGES.txt (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/CHANGES.txt Tue Feb 22 01:00:39 2011
@@ -80,12 +80,20 @@ Bug Fixes
 
 * SOLR-2330: solrconfig.xml files in example-DIH are broken. (Matt Parker, koji)
 
+* SOLR-2116: Fix TikaConfig classloader bug in TikaEntityProcessor
+  (Martijn van Groningen via hossman)
+  
+
 Other Changes
 ----------------------
 
 * SOLR-1821: Fix TimeZone-dependent test failure in TestEvaluatorBag.
   (Chris Male via rmuir)
 
+* SOLR-2367: Reduced noise in test output by ensuring the properties file can be written.
+  (Gunnlaugur Thor Briem via rmuir)
+
+
 Build
 ----------------------
 

Added: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/README.txt?rev=1073192&view=auto
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/README.txt (added)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/README.txt Tue Feb 22 01:00:39 2011
@@ -0,0 +1,7 @@
+Although Solr strives to be agnostic of the Locale where the server is
+running, some code paths in DataImportHandler are known to depend on the
+System default Locale, Timezone, or Charset.  It is recommended that when
+running Solr you set the following system properties:
+  -Duser.language=xx -Duser.country=YY -Duser.timezone=ZZZ
+
+where xx, YY, and ZZZ are consistent with any database server's configuration.

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/build.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/build.xml (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/build.xml Tue Feb 22 01:00:39 2011
@@ -56,7 +56,7 @@
 	  <pathelement path="target/classes" />
   	<pathelement path="target/test-classes" />
     <pathelement location="${solr-path}/build/tests"/> <!-- include solr test code -->
-    <pathelement location="${solr-path}/../lucene/build/classes/test" />  <!-- include some lucene test code -->
+    <pathelement location="${solr-path}/../lucene/build/classes/test-framework" />  <!-- include some lucene test code -->
     <pathelement path="${java.class.path}"/>
   </path>
 
@@ -68,7 +68,7 @@
   	<pathelement path="target/test-classes" />
   	<pathelement path="target/extras/test-classes" />
     <pathelement location="${solr-path}/build/tests"/> <!-- include solr test code -->
-    <pathelement location="${solr-path}/../lucene/build/classes/test" />  <!-- include some lucene test code -->
+    <pathelement location="${solr-path}/../lucene/build/classes/test-framework" />  <!-- include some lucene test code -->
     <pathelement path="${java.class.path}"/>
   </path>
 	
@@ -276,7 +276,7 @@
   		<fileset dir="src/main/webapp" includes="**" />
   	</copy>
   	<mkdir dir="../../build/web/WEB-INF/lib"/>
-  	<copy file="target/${fullnamever}.jar" todir="${solr-path}/build/web/WEB-INF/lib"></copy>
+  	<!--<copy file="target/${fullnamever}.jar" todir="${solr-path}/build/web/WEB-INF/lib"></copy>-->
   	<copy file="target/${fullnamever}.jar" todir="${solr-path}/dist"></copy>
   	<copy file="target/apache-${ant.project.name}-extras-${version}.jar" todir="${solr-path}/dist"></copy>
   </target>

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/MailEntityProcessor.java Tue Feb 22 01:00:39 2011
@@ -51,6 +51,7 @@ public class MailEntityProcessor extends
     public SearchTerm getCustomSearch(Folder folder);
   }
 
+  @Override
   public void init(Context context) {
     super.init(context);
     // set attributes using  XXX getXXXFromContext(attribute, defualtValue);
@@ -95,6 +96,7 @@ public class MailEntityProcessor extends
     logConfig();
   }
 
+  @Override
   public Map<String, Object> nextRow() {
     Message mail;
     Map<String, Object> row = null;

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/main/java/org/apache/solr/handler/dataimport/TikaEntityProcessor.java Tue Feb 22 01:00:39 2011
@@ -17,15 +17,11 @@
 package org.apache.solr.handler.dataimport;
 
 import org.apache.commons.io.IOUtils;
-import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
-import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
-import static org.apache.solr.handler.dataimport.DataImporter.COLUMN;
-import static org.apache.solr.handler.dataimport.XPathEntityProcessor.URL;
 import org.apache.tika.config.TikaConfig;
 import org.apache.tika.metadata.Metadata;
 import org.apache.tika.parser.AutoDetectParser;
-import org.apache.tika.parser.Parser;
 import org.apache.tika.parser.ParseContext;
+import org.apache.tika.parser.Parser;
 import org.apache.tika.sax.BodyContentHandler;
 import org.apache.tika.sax.ContentHandlerDecorator;
 import org.apache.tika.sax.XHTMLContentHandler;
@@ -47,6 +43,11 @@ import java.io.StringWriter;
 import java.io.Writer;
 import java.util.HashMap;
 import java.util.Map;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+import static org.apache.solr.handler.dataimport.DataImporter.COLUMN;
+import static org.apache.solr.handler.dataimport.XPathEntityProcessor.URL;
 /**
  * <p>An implementation of {@link EntityProcessor} which reads data from rich docs
  * using <a href="http://tika.apache.org/">Apache Tika</a>
@@ -65,19 +66,20 @@ public class TikaEntityProcessor extends
 
   @Override
   protected void firstInit(Context context) {
-    String tikaConfigFile = context.getResolvedEntityAttribute("tikaConfig");
-    if (tikaConfigFile == null) {
-      tikaConfig = TikaConfig.getDefaultConfig();
-    } else {
-      File configFile = new File(tikaConfigFile);
-      if (!configFile.isAbsolute()) {
-        configFile = new File(context.getSolrCore().getResourceLoader().getConfigDir(), tikaConfigFile);
-      }
-      try {
+    try {
+      String tikaConfigFile = context.getResolvedEntityAttribute("tikaConfig");
+      if (tikaConfigFile == null) {
+        ClassLoader classLoader = context.getSolrCore().getResourceLoader().getClassLoader();
+        tikaConfig = new TikaConfig(classLoader);
+      } else {
+        File configFile = new File(tikaConfigFile);
+        if (!configFile.isAbsolute()) {
+          configFile = new File(context.getSolrCore().getResourceLoader().getConfigDir(), tikaConfigFile);
+        }
         tikaConfig = new TikaConfig(configFile);
-      } catch (Exception e) {
-        wrapAndThrow (SEVERE, e,"Unable to load Tika Config");
       }
+    } catch (Exception e) {
+      wrapAndThrow (SEVERE, e,"Unable to load Tika Config");
     }
 
     format = context.getResolvedEntityAttribute("format");
@@ -92,6 +94,7 @@ public class TikaEntityProcessor extends
     done = false;
   }
 
+  @Override
   public Map<String, Object> nextRow() {
     if(done) return null;
     Map<String, Object> row = new HashMap<String, Object>();

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestMailEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestMailEntityProcessor.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestMailEntityProcessor.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestMailEntityProcessor.java Tue Feb 22 01:00:39 2011
@@ -191,18 +191,22 @@ public class TestMailEntityProcessor ext
       super(null, ".", null);
     }
 
+    @Override
     public boolean upload(SolrInputDocument doc) {
       return docs.add(doc);
     }
 
+    @Override
     public void log(int event, String name, Object row) {
       // Do nothing
     }
 
+    @Override
     public void doDeleteAll() {
       deleteAllCalled = Boolean.TRUE;
     }
 
+    @Override
     public void commit(boolean b) {
       commitCalled = Boolean.TRUE;
     }

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestTikaEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestTikaEntityProcessor.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestTikaEntityProcessor.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/java/org/apache/solr/handler/dataimport/TestTikaEntityProcessor.java Tue Feb 22 01:00:39 2011
@@ -17,6 +17,7 @@
 package org.apache.solr.handler.dataimport;
 
 import org.junit.BeforeClass;
+import org.junit.Test;
 
 /**Testcase for TikaEntityProcessor
  * @version $Id$
@@ -25,9 +26,10 @@ import org.junit.BeforeClass;
 public class TestTikaEntityProcessor extends AbstractDataImportHandlerTestCase {
   @BeforeClass
   public static void beforeClass() throws Exception {
-    initCore("dataimport-solrconfig.xml", "dataimport-schema-no-unique-key.xml", "solr-dihextras");
+    initCore("dataimport-solrconfig.xml", "dataimport-schema-no-unique-key.xml", getFile("solr-dihextras").getAbsolutePath());
   }
 
+  @Test
   public void testIndexingWithTikaEntityProcessor() throws Exception {
     String conf =
             "<dataConfig>" +
@@ -35,12 +37,17 @@ public class TestTikaEntityProcessor ext
                     "  <document>" +
                     "    <entity processor=\"TikaEntityProcessor\" url=\"" + getFile("solr-word.pdf").getAbsolutePath() + "\" >" +
                     "      <field column=\"Author\" meta=\"true\" name=\"author\"/>" +
-                    "      <field column=\"title\" meta=\"true\" name=\"docTitle\"/>" +
+                    "      <field column=\"title\" meta=\"true\" name=\"title\"/>" +
                     "      <field column=\"text\"/>" +
                     "     </entity>" +
                     "  </document>" +
                     "</dataConfig>";
     runFullImport(conf);
-    assertQ(req("*:*"), "//*[@numFound='1']");
+    assertQ(req("*:*")
+            ,"//*[@numFound='1']"
+            ,"//str[@name='author'][.='Grant Ingersoll']"
+            ,"//str[@name='title'][.='solr-word']"
+            ,"//str[@name='text']"
+            );
   }
 }

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/resources/solr-dihextras/conf/dataimport-solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/resources/solr-dihextras/conf/dataimport-solrconfig.xml?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/resources/solr-dihextras/conf/dataimport-solrconfig.xml (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/extras/test/resources/solr-dihextras/conf/dataimport-solrconfig.xml Tue Feb 22 01:00:39 2011
@@ -18,19 +18,11 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
-  <!-- Set this to 'false' if you want solr to continue working after it has 
-       encountered an severe configuration error.  In a production environment, 
-       you may want solr to keep working even if one handler is mis-configured.
-
-       You may also set this to false using by setting the system property:
-         -Dsolr.abortOnConfigurationError=false
-     -->
-  <abortOnConfigurationError>${solr.abortOnConfigurationError:true}</abortOnConfigurationError>
 
   <!-- Used to specify an alternate directory to hold all index data
        other than the default ./data under the Solr home.
        If replication is in use, this should match the replication configuration. -->
-       <dataDir>${solr.data.dir:./solr/data}</dataDir>
+       <dataDir>${solr.data.dir:}</dataDir>
 
 
   <indexDefaults>

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinContentStreamDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinContentStreamDataSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinContentStreamDataSource.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinContentStreamDataSource.java Tue Feb 22 01:00:39 2011
@@ -39,10 +39,12 @@ public class BinContentStreamDataSource 
   private InputStream in;
 
 
+  @Override
   public void init(Context context, Properties initProps) {
     this.context = (ContextImpl) context;
   }
 
+  @Override
   public InputStream getData(String query) {
      contentStream = context.getDocBuilder().requestParameters.contentStream;
     if (contentStream == null)
@@ -55,6 +57,7 @@ public class BinContentStreamDataSource 
     }
   }
 
+  @Override
   public void close() {
      if (contentStream != null) {
       try {

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinFileDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinFileDataSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinFileDataSource.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinFileDataSource.java Tue Feb 22 01:00:39 2011
@@ -43,10 +43,12 @@ import java.util.Properties;
 
 public class BinFileDataSource extends DataSource<InputStream>{
    protected String basePath;
+  @Override
   public void init(Context context, Properties initProps) {
      basePath = initProps.getProperty(FileDataSource.BASE_PATH);
   }
 
+  @Override
   public InputStream getData(String query) {
     File f = FileDataSource.getFile(basePath,query);
     try {
@@ -57,6 +59,7 @@ public class BinFileDataSource extends D
     }
   }
 
+  @Override
   public void close() {
 
   }

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinURLDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinURLDataSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinURLDataSource.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/BinURLDataSource.java Tue Feb 22 01:00:39 2011
@@ -49,6 +49,7 @@ public class BinURLDataSource extends Da
 
   public BinURLDataSource() { }
 
+  @Override
   public void init(Context context, Properties initProps) {
       this.context = context;
     this.initProps = initProps;
@@ -72,6 +73,7 @@ public class BinURLDataSource extends Da
     }
   }
 
+  @Override
   public InputStream getData(String query) {
     URL url = null;
     try {
@@ -89,6 +91,7 @@ public class BinURLDataSource extends Da
     }
   }
 
+  @Override
   public void close() { }
 
   private String getInitPropWithReplacements(String propertyName) {

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/CachedSqlEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/CachedSqlEntityProcessor.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/CachedSqlEntityProcessor.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/CachedSqlEntityProcessor.java Tue Feb 22 01:00:39 2011
@@ -38,6 +38,7 @@ import java.util.Map;
 public class CachedSqlEntityProcessor extends SqlEntityProcessor {
   private boolean isFirst;
 
+  @Override
   @SuppressWarnings("unchecked")
   public void init(Context context) {
     super.init(context);
@@ -45,6 +46,7 @@ public class CachedSqlEntityProcessor ex
     isFirst = true;
   }
 
+  @Override
   public Map<String, Object> nextRow() {
     if (dataSourceRowCache != null)
       return getFromRowCacheTransformed();
@@ -60,6 +62,7 @@ public class CachedSqlEntityProcessor ex
 
   }
 
+  @Override
   protected List<Map<String, Object>> getAllNonCachedRows() {
     List<Map<String, Object>> rows = new ArrayList<Map<String, Object>>();
     String q = getQuery();

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ClobTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ClobTransformer.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ClobTransformer.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ClobTransformer.java Tue Feb 22 01:00:39 2011
@@ -37,6 +37,7 @@ import java.util.Map;
  * @since solr 1.4
  */
 public class ClobTransformer extends Transformer {
+  @Override
   public Object transformRow(Map<String, Object> aRow, Context context) {
     for (Map<String, String> map : context.getAllEntityFields()) {
       if (!TRUE.equals(map.get(CLOB))) continue;

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContentStreamDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContentStreamDataSource.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContentStreamDataSource.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContentStreamDataSource.java Tue Feb 22 01:00:39 2011
@@ -39,10 +39,12 @@ public class ContentStreamDataSource ext
   private ContentStream contentStream;
   private Reader reader;
 
+  @Override
   public void init(Context context, Properties initProps) {
     this.context = (ContextImpl) context;
   }
 
+  @Override
   public Reader getData(String query) {
     contentStream = context.getDocBuilder().requestParameters.contentStream;
     if (contentStream == null)
@@ -55,6 +57,7 @@ public class ContentStreamDataSource ext
     }
   }
 
+  @Override
   public void close() {
     if (contentStream != null) {
       try {

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContextImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContextImpl.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContextImpl.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ContextImpl.java Tue Feb 22 01:00:39 2011
@@ -71,22 +71,27 @@ public class ContextImpl extends Context
     parent = parentContext;
   }
 
+  @Override
   public String getEntityAttribute(String name) {
     return entity == null ? null : entity.allAttributes.get(name);
   }
 
+  @Override
   public String getResolvedEntityAttribute(String name) {
     return entity == null ? null : resolver.replaceTokens(entity.allAttributes.get(name));
   }
 
+  @Override
   public List<Map<String, String>> getAllEntityFields() {
     return entity == null ? Collections.EMPTY_LIST : entity.allFieldsList;
   }
 
+  @Override
   public VariableResolver getVariableResolver() {
     return resolver;
   }
 
+  @Override
   public DataSource getDataSource() {
     if (ds != null) return ds;
     if(entity == null) return  null;
@@ -101,26 +106,32 @@ public class ContextImpl extends Context
     return entity.dataSrc;
   }
 
+  @Override
   public DataSource getDataSource(String name) {
     return dataImporter.getDataSourceInstance(entity, name, this);
   }
 
+  @Override
   public boolean isRootEntity() {
     return entity.isDocRoot;
   }
 
+  @Override
   public String currentProcess() {
     return currProcess;
   }
 
+  @Override
   public Map<String, Object> getRequestParameters() {
     return requestParams;
   }
 
+  @Override
   public EntityProcessor getEntityProcessor() {
     return entity == null ? null : entity.processor;
   }
 
+  @Override
   public void setSessionAttribute(String name, Object val, String scope) {
     if(name == null) return;
     if (Context.SCOPE_ENTITY.equals(scope)) {
@@ -148,6 +159,7 @@ public class ContextImpl extends Context
     else entitySession.put(name, val);
   }
 
+  @Override
   public Object getSessionAttribute(String name, String scope) {
     if (Context.SCOPE_ENTITY.equals(scope)) {
       if (entitySession == null)
@@ -166,6 +178,7 @@ public class ContextImpl extends Context
     return null;
   }
 
+  @Override
   public Context getParentContext() {
     return parent;
   }
@@ -187,15 +200,18 @@ public class ContextImpl extends Context
   }
 
 
+  @Override
   public SolrCore getSolrCore() {
     return dataImporter == null ? null : dataImporter.getCore();
   }
 
 
+  @Override
   public Map<String, Object> getStats() {
     return docBuilder != null ? docBuilder.importStatistics.getStatsSnapshot() : Collections.<String, Object>emptyMap();
   }
 
+  @Override
   public String getScript() {
     if(dataImporter != null) {
       DataConfig.Script script = dataImporter.getConfig().script;
@@ -204,6 +220,7 @@ public class ContextImpl extends Context
     return null;
   }
 
+  @Override
   public String getScriptLanguage() {
     if (dataImporter != null) {
       DataConfig.Script script = dataImporter.getConfig().script;
@@ -212,12 +229,14 @@ public class ContextImpl extends Context
     return null;
   }
 
+  @Override
   public void deleteDoc(String id) {
     if(docBuilder != null){
       docBuilder.writer.deleteDoc(id);
     }
   }
 
+  @Override
   public void deleteDocByQuery(String query) {
     if(docBuilder != null){
       docBuilder.writer.deleteByQuery(query);
@@ -227,10 +246,12 @@ public class ContextImpl extends Context
   DocBuilder getDocBuilder(){
     return docBuilder;
   }
+  @Override
   public Object resolve(String var) {
     return resolver.resolve(var);
   }
 
+  @Override
   public String replaceTokens(String template) {
     return resolver.replaceTokens(template);
   }

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataConfig.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataConfig.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataConfig.java Tue Feb 22 01:00:39 2011
@@ -214,6 +214,7 @@ public class DataConfig {
 
 
     public Map<String, String> allAttributes = new HashMap<String, String>() {
+      @Override
       public String put(String key, String value) {
         if (super.containsKey(key))
           return super.get(key);

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImporter.java Tue Feb 22 01:00:39 2011
@@ -17,6 +17,7 @@
 
 package org.apache.solr.handler.dataimport;
 
+import org.apache.solr.common.SolrException;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
@@ -336,7 +337,7 @@ public class DataImporter {
       if (!requestParams.debug)
         cumulativeStatistics.add(docBuilder.importStatistics);
     } catch (Throwable t) {
-      LOG.error("Full Import failed", t);
+      SolrException.log(LOG, "Full Import failed", t);
       docBuilder.rollback();
     } finally {
       setStatus(Status.IDLE);
@@ -423,6 +424,7 @@ public class DataImporter {
   }
 
   static final ThreadLocal<AtomicLong> QUERY_COUNT = new ThreadLocal<AtomicLong>() {
+    @Override
     protected AtomicLong initialValue() {
       return new AtomicLong();
     }

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DateFormatTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DateFormatTransformer.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DateFormatTransformer.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DateFormatTransformer.java Tue Feb 22 01:00:39 2011
@@ -45,6 +45,7 @@ public class DateFormatTransformer exten
   private static final Logger LOG = LoggerFactory
           .getLogger(DateFormatTransformer.class);
 
+  @Override
   @SuppressWarnings("unchecked")
   public Object transformRow(Map<String, Object> aRow, Context context) {
 

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DebugLogger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DebugLogger.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DebugLogger.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DebugLogger.java Tue Feb 22 01:00:39 2011
@@ -60,6 +60,7 @@ class DebugLogger {
     output = new NamedList();
     debugStack = new Stack<DebugInfo>() {
 
+      @Override
       public DebugInfo pop() {
         if (size() == 1)
           throw new DataImportHandlerException(
@@ -169,14 +170,17 @@ class DebugLogger {
 
   DataSource wrapDs(final DataSource ds) {
     return new DataSource() {
+      @Override
       public void init(Context context, Properties initProps) {
         ds.init(context, initProps);
       }
 
+      @Override
       public void close() {
         ds.close();
       }
 
+      @Override
       public Object getData(String query) {
         writer.log(SolrWriter.ENTITY_META, "query", query);
         long start = System.currentTimeMillis();
@@ -203,6 +207,7 @@ class DebugLogger {
 
   Transformer wrapTransformer(final Transformer t) {
     return new Transformer() {
+      @Override
       public Object transformRow(Map<String, Object> row, Context context) {
         writer.log(SolrWriter.PRE_TRANSFORMER_ROW, null, row);
         String tName = getTransformerName(t);

Modified: lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java (original)
+++ lucene/dev/branches/realtime_search/solr/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DocBuilder.java Tue Feb 22 01:00:39 2011
@@ -17,6 +17,7 @@
 
 package org.apache.solr.handler.dataimport;
 
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.core.SolrCore;
 import static org.apache.solr.handler.dataimport.SolrWriter.LAST_INDEX_KEY;
@@ -139,6 +140,7 @@ public class DocBuilder {
     document = dataImporter.getConfig().document;
     final AtomicLong startTime = new AtomicLong(System.currentTimeMillis());
     statusMessages.put(TIME_ELAPSED, new Object() {
+      @Override
       public String toString() {
         return getTimeElapsedSince(startTime.get());
       }
@@ -317,7 +319,11 @@ public class DocBuilder {
       String keyName = root.isDocRoot ? root.getPk() : root.getSchemaPk();
       Object key = map.get(keyName);
       if(key == null) {
-        LOG.warn("no key was available for deleteted pk query. keyName = " + keyName);
+        keyName = findMatchingPkColumn(keyName, map);
+        key = map.get(keyName);
+      }
+      if(key == null) {
+        LOG.warn("no key was available for deleted pk query. keyName = " + keyName);
         continue;
       }
       writer.deleteDoc(key);
@@ -485,7 +491,7 @@ public class DocBuilder {
                 importStatistics.skipDocCount.getAndIncrement();
                 exception = null;//should not propogate up
               } else {
-                LOG.error("Exception while processing: "
+                SolrException.log(LOG, "Exception while processing: "
                         + entity.name + " document : " + docWrapper, dihe);
               }
               if (dihe.getErrCode() == DataImportHandlerException.SEVERE)
@@ -644,7 +650,7 @@ public class DocBuilder {
               importStatistics.skipDocCount.getAndIncrement();
               doc = null;
             } else {
-              LOG.error("Exception while processing: "
+              SolrException.log(LOG, "Exception while processing: "
                       + entity.name + " document : " + doc, e);
             }
             if (e.getErrCode() == DataImportHandlerException.SEVERE)
@@ -815,6 +821,28 @@ public class DocBuilder {
     return entity.processor = new EntityProcessorWrapper(entityProcessor, this);
   }
 
+  private String findMatchingPkColumn(String pk, Map<String, Object> row) {
+    if (row.containsKey(pk))
+      throw new IllegalArgumentException(
+        String.format("deltaQuery returned a row with null for primary key %s", pk));
+    String resolvedPk = null;
+    for (String columnName : row.keySet()) {
+      if (columnName.endsWith("." + pk) || pk.endsWith("." + columnName)) {
+        if (resolvedPk != null)
+          throw new IllegalArgumentException(
+            String.format(
+              "deltaQuery has more than one column (%s and %s) that might resolve to declared primary key pk='%s'",
+              resolvedPk, columnName, pk));
+        resolvedPk = columnName;
+      }
+    }
+    if (resolvedPk == null)
+      throw new IllegalArgumentException(
+        String.format("deltaQuery has no column to resolve to declared primary key pk='%s'", pk));
+    LOG.info(String.format("Resolving deltaQuery column '%s' to match entity's declared pk '%s'", resolvedPk, pk));
+    return resolvedPk;
+  }
+
   /**
    * <p> Collects unique keys of all Solr documents for whom one or more source tables have been changed since the last
    * indexed time. </p> <p> Note: In our definition, unique key of Solr document is the primary key of the top level
@@ -851,13 +879,20 @@ public class DocBuilder {
     Map<String, Map<String, Object>> deltaSet = new HashMap<String, Map<String, Object>>();
     LOG.info("Running ModifiedRowKey() for Entity: " + entity.name);
     //get the modified rows in this entity
+    String pk = entity.getPk();
     while (true) {
       Map<String, Object> row = entityProcessor.nextModifiedRowKey();
 
       if (row == null)
         break;
 
-      deltaSet.put(row.get(entity.getPk()).toString(), row);
+      Object pkValue = row.get(pk);
+      if (pkValue == null) {
+        pk = findMatchingPkColumn(pk, row);
+        pkValue = row.get(pk);
+      }
+
+      deltaSet.put(pkValue.toString(), row);
       importStatistics.rowsCount.incrementAndGet();
       // check for abort
       if (stop.get())
@@ -872,8 +907,14 @@ public class DocBuilder {
 
       deletedSet.add(row);
       
+      Object pkValue = row.get(pk);
+      if (pkValue == null) {
+        pk = findMatchingPkColumn(pk, row);
+        pkValue = row.get(pk);
+      }
+
       // Remove deleted rows from the delta rows
-      String deletedRowPk = row.get(entity.getPk()).toString();
+      String deletedRowPk = pkValue.toString();
       if (deltaSet.containsKey(deletedRowPk)) {
         deltaSet.remove(deletedRowPk);
       }