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 [19/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/solr/src/java/org/apache/solr/schema/PointType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/PointType.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/PointType.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/PointType.java Mon Mar 28 10:50:28 2011
@@ -68,7 +68,8 @@ public class PointType extends Coordinat
   }
 
   @Override
-  public Fieldable[] createFields(SchemaField field, String externalVal, float boost) {
+  public Fieldable[] createFields(SchemaField field, Object value, float boost) {
+    String externalVal = value.toString();
     String[] point = new String[0];
     try {
       point = DistanceUtils.parsePoint(null, externalVal, dimension);
@@ -112,7 +113,7 @@ public class PointType extends Coordinat
    *
    */
   @Override
-  public Fieldable createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, Object value, float boost) {
     throw new UnsupportedOperationException("PointType uses multiple fields.  field=" + field.getName());
   }
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/SchemaField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/SchemaField.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/SchemaField.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/SchemaField.java Mon Mar 28 10:50:28 2011
@@ -93,16 +93,16 @@ public final class SchemaField extends F
   boolean isBinary() { return (properties & BINARY)!=0; }
 
 
-  public Fieldable createField(String val, float boost) {
+  public Fieldable createField(Object val, float boost) {
     return type.createField(this,val,boost);
   }
   
-  public Fieldable[] createFields(String val, float boost) {
+  public Fieldable[] createFields(Object val, float boost) {
     return type.createFields(this,val,boost);
   }
 
   /**
-   * If true, then use {@link #createFields(String, float)}, else use {@link #createField} to save an extra allocation
+   * If true, then use {@link #createFields(Object, float)}, else use {@link #createField} to save an extra allocation
    * @return true if this field is a poly field
    */
   public boolean isPolyField(){
@@ -150,7 +150,6 @@ public final class SchemaField extends F
                               "can not sort on multivalued field: " 
                               + getName());
     }
-    
   }
 
   /** 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TextField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TextField.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TextField.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TextField.java Mon Mar 28 10:50:28 2011
@@ -84,6 +84,15 @@ public class TextField extends FieldType
     return ByteUtils.UTF8toUTF16(term);
   }
 
+  @Override
+  public void setAnalyzer(Analyzer analyzer) {
+    this.analyzer = analyzer;
+  }
+
+  @Override
+  public void setQueryAnalyzer(Analyzer analyzer) {
+    this.queryAnalyzer = analyzer;
+  }
 
   static Query parseFieldQuery(QParser parser, Analyzer analyzer, String field, String queryText) {
     int phraseSlop = 0;

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieDateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieDateField.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieDateField.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieDateField.java Mon Mar 28 10:50:28 2011
@@ -79,7 +79,18 @@ public class TrieDateField extends DateF
   @Override
   public SortField getSortField(SchemaField field, boolean top) {
     field.checkSortability();
-    return new SortField(new LongValuesCreator( field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER, CachedArrayCreator.CACHE_VALUES_AND_BITS ), top);
+
+    int flags = CachedArrayCreator.CACHE_VALUES_AND_BITS;
+    boolean sortMissingLast  = field.sortMissingLast();
+    boolean sortMissingFirst = field.sortMissingFirst();
+
+    Object missingValue = null;
+    if( sortMissingLast ) {
+      missingValue = top ? Long.MIN_VALUE : Long.MAX_VALUE;
+    } else if( sortMissingFirst ) {
+      missingValue = top ? Long.MAX_VALUE : Long.MIN_VALUE;
+    }
+    return new SortField(new LongValuesCreator(field.getName(), FieldCache.NUMERIC_UTILS_LONG_PARSER, flags), top).setMissingValue(missingValue);
   }
 
   @Override
@@ -157,7 +168,7 @@ public class TrieDateField extends DateF
   }
 
   @Override
-  public Fieldable createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, Object value, float boost) {
     boolean indexed = field.indexed();
     boolean stored = field.stored();
 
@@ -172,7 +183,10 @@ public class TrieDateField extends DateF
     byte[] arr=null;
     TokenStream ts=null;
 
-    long time = super.parseMath(null, externalVal).getTime();
+    long time = (value instanceof Date) 
+      ? ((Date)value).getTime() 
+      : super.parseMath(null, value.toString()).getTime();
+      
     if (stored) arr = TrieField.toArr(time);
     if (indexed) ts = new NumericTokenStream(ps).setLongValue(time);
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieField.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieField.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/schema/TrieField.java Mon Mar 28 10:50:28 2011
@@ -127,8 +127,8 @@ public class TrieField extends FieldType
 
     int flags = CachedArrayCreator.CACHE_VALUES_AND_BITS;
     Object missingValue = null;
-    boolean sortMissingLast  = on( SORT_MISSING_LAST,  properties );
-    boolean sortMissingFirst = on( SORT_MISSING_FIRST, properties );
+    boolean sortMissingLast  = field.sortMissingLast();
+    boolean sortMissingFirst = field.sortMissingFirst();
     
     switch (type) {
       case INTEGER:
@@ -482,7 +482,7 @@ public class TrieField extends FieldType
   }
 
   @Override
-  public Fieldable createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, Object value, float boost) {
     boolean indexed = field.indexed();
     boolean stored = field.stored();
 
@@ -500,27 +500,37 @@ public class TrieField extends FieldType
 
     switch (type) {
       case INTEGER:
-        int i = Integer.parseInt(externalVal);
+        int i = (value instanceof Number)
+          ? ((Number)value).intValue()
+          : Integer.parseInt(value.toString());
         if (stored) arr = toArr(i);
         if (indexed) ts = new NumericTokenStream(ps).setIntValue(i);
         break;
       case FLOAT:
-        float f = Float.parseFloat(externalVal);
+        float f = (value instanceof Number)
+          ? ((Number)value).floatValue()
+          : Float.parseFloat(value.toString());
         if (stored) arr = toArr(f);
         if (indexed) ts = new NumericTokenStream(ps).setFloatValue(f);
         break;
       case LONG:
-        long l = Long.parseLong(externalVal);
+        long l = (value instanceof Number)
+          ? ((Number)value).longValue()
+          : Long.parseLong(value.toString());
         if (stored) arr = toArr(l);
         if (indexed) ts = new NumericTokenStream(ps).setLongValue(l);
         break;
       case DOUBLE:
-        double d = Double.parseDouble(externalVal);
+        double d = (value instanceof Number)
+          ? ((Number)value).doubleValue()
+          : Double.parseDouble(value.toString());
         if (stored) arr = toArr(d);
         if (indexed) ts = new NumericTokenStream(ps).setDoubleValue(d);
         break;
       case DATE:
-        long time = dateField.parseMath(null, externalVal).getTime();
+        long time = (value instanceof Date)
+          ? ((Date)value).getTime()
+          : dateField.parseMath(null, value.toString()).getTime();
         if (stored) arr = toArr(time);
         if (indexed) ts = new NumericTokenStream(ps).setLongValue(time);
         break;

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/QueryParsing.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/QueryParsing.java Mon Mar 28 10:50:28 2011
@@ -398,8 +398,8 @@ public class QueryParsing {
       String fname = q.getField();
       FieldType ft = writeFieldName(fname, schema, out, flags);
       out.append(q.includesLower() ? '[' : '{');
-      String lt = q.getLowerTerm();
-      String ut = q.getUpperTerm();
+      String lt = q.getLowerTerm().utf8ToString();
+      String ut = q.getUpperTerm().utf8ToString();
       if (lt == null) {
         out.append('*');
       } else {
@@ -582,7 +582,7 @@ public class QueryParsing {
 
     boolean opt(char ch) {
       eatws();
-      if (val.charAt(pos) == ch) {
+      if (pos < end && val.charAt(pos) == ch) {
         pos++;
         return true;
       }
@@ -693,7 +693,29 @@ public class QueryParsing {
         pos++;
         while (pos < end) {
           ch = val.charAt(pos);
-          if (!Character.isJavaIdentifierPart(ch) && ch != '.') {
+          if (!Character.isJavaIdentifierPart(ch) && ch != '.' && ch != ':') {
+            break;
+          }
+          pos++;
+        }
+        return val.substring(id_start, pos);
+      }
+
+      if (errMessage != null) {
+        throw new ParseException(errMessage + " at pos " + pos + " str='" + val + "'");
+      }
+      return null;
+    }
+
+    public String getGlobbedId(String errMessage) throws ParseException {
+      eatws();
+      int id_start = pos;
+      char ch;
+      if (pos < end && (ch = val.charAt(pos)) != '$' && (Character.isJavaIdentifierStart(ch) || ch=='?' || ch=='*')) {
+        pos++;
+        while (pos < end) {
+          ch = val.charAt(pos);
+          if (!(Character.isJavaIdentifierPart(ch) || ch=='?' || ch=='*') && ch != '.') {
             break;
           }
           pos++;

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/ValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/ValueSourceParser.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/ValueSourceParser.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/search/ValueSourceParser.java Mon Mar 28 10:50:28 2011
@@ -583,7 +583,7 @@ public abstract class ValueSourceParser 
     if (ft == null) ft = new StrField();
 
     if (ft instanceof TextField) {
-      // need to do analyisis on the term
+      // need to do analysis on the term
       String indexedVal = tinfo.val;
       Query q = ft.getFieldQuery(fp, fp.getReq().getSchema().getFieldOrNull(tinfo.field), tinfo.val);
       if (q instanceof TermQuery) {

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Lookup.java Mon Mar 28 10:50:28 2011
@@ -39,7 +39,7 @@ public abstract class Lookup {
   public static final class LookupPriorityQueue extends PriorityQueue<LookupResult> {
     
     public LookupPriorityQueue(int size) {
-      initialize(size);
+      super(size);
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/Suggester.java Mon Mar 28 10:50:28 2011
@@ -83,6 +83,8 @@ public class Suggester extends SolrSpell
     if (lookupImpl == null) {
       lookupImpl = JaspellLookup.class.getName();
     }
+    lookup = (Lookup) core.getResourceLoader().newInstance(lookupImpl);
+    lookup.init(config, core);
     String store = (String)config.get(STORE_DIR);
     if (store != null) {
       storeDir = new File(store);
@@ -91,6 +93,13 @@ public class Suggester extends SolrSpell
       }
       if (!storeDir.exists()) {
         storeDir.mkdirs();
+      } else {
+        // attempt reload of the stored lookup
+        try {
+          lookup.load(storeDir);
+        } catch (IOException e) {
+          LOG.warn("Loading stored lookup data failed", e);
+        }
       }
     }
     return name;
@@ -107,17 +116,17 @@ public class Suggester extends SolrSpell
         dictionary = new FileDictionary(new InputStreamReader(
                 core.getResourceLoader().openResource(sourceLocation), "UTF-8"));
       } catch (UnsupportedEncodingException e) {
-        e.printStackTrace();
+        // should not happen
+        LOG.error("should not happen", e);
       }
     }
-    lookup = (Lookup) core.getResourceLoader().newInstance(lookupImpl);
     try {
       lookup.build(dictionary);
       if (storeDir != null) {
         lookup.store(storeDir);
       }
     } catch (Exception e) {
-      e.printStackTrace();
+      LOG.error("Error while building or storing Suggester data", e);
     }
   }
 

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellLookup.java Mon Mar 28 10:50:28 2011
@@ -1,6 +1,10 @@
 package org.apache.solr.spelling.suggest.jaspell;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -9,6 +13,7 @@ import org.apache.solr.common.util.Named
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.spelling.suggest.Lookup;
 import org.apache.solr.spelling.suggest.UnsortedTermFreqIteratorWrapper;
+import org.apache.solr.spelling.suggest.jaspell.JaspellTernarySearchTrie.TSTNode;
 import org.apache.solr.util.SortedIterator;
 import org.apache.solr.util.TermFreqIterator;
 import org.slf4j.Logger;
@@ -16,7 +21,7 @@ import org.slf4j.LoggerFactory;
 
 public class JaspellLookup extends Lookup {
   private static final Logger LOG = LoggerFactory.getLogger(JaspellLookup.class);
-  JaspellTernarySearchTrie trie;
+  JaspellTernarySearchTrie trie = new JaspellTernarySearchTrie();
   private boolean usePrefix = true;
   private int editDistance = 2;
 
@@ -89,14 +94,89 @@ public class JaspellLookup extends Looku
     return res;
   }
 
+  public static final String FILENAME = "jaspell.dat";
+  private static final byte LO_KID = 0x01;
+  private static final byte EQ_KID = 0x02;
+  private static final byte HI_KID = 0x04;
+  private static final byte HAS_VALUE = 0x08;
+ 
+  
   @Override
   public boolean load(File storeDir) throws IOException {
-    return false;
+    File data = new File(storeDir, FILENAME);
+    if (!data.exists() || !data.canRead()) {
+      return false;
+    }
+    DataInputStream in = new DataInputStream(new FileInputStream(data));
+    TSTNode root = trie.new TSTNode('\0', null);
+    try {
+      readRecursively(in, root);
+      trie.setRoot(root);
+    } finally {
+      in.close();
+    }
+    return true;
+  }
+  
+  private void readRecursively(DataInputStream in, TSTNode node) throws IOException {
+    node.splitchar = in.readChar();
+    byte mask = in.readByte();
+    if ((mask & HAS_VALUE) != 0) {
+      node.data = new Float(in.readFloat());
+    }
+    if ((mask & LO_KID) != 0) {
+      TSTNode kid = trie.new TSTNode('\0', node);
+      node.relatives[TSTNode.LOKID] = kid;
+      readRecursively(in, kid);
+    }
+    if ((mask & EQ_KID) != 0) {
+      TSTNode kid = trie.new TSTNode('\0', node);
+      node.relatives[TSTNode.EQKID] = kid;
+      readRecursively(in, kid);
+    }
+    if ((mask & HI_KID) != 0) {
+      TSTNode kid = trie.new TSTNode('\0', node);
+      node.relatives[TSTNode.HIKID] = kid;
+      readRecursively(in, kid);
+    }
   }
 
   @Override
   public boolean store(File storeDir) throws IOException {
-    return false;    
+    if (!storeDir.exists() || !storeDir.isDirectory() || !storeDir.canWrite()) {
+      return false;
+    }
+    TSTNode root = trie.getRoot();
+    if (root == null) { // empty tree
+      return false;
+    }
+    File data = new File(storeDir, FILENAME);
+    DataOutputStream out = new DataOutputStream(new FileOutputStream(data));
+    try {
+      writeRecursively(out, root);
+      out.flush();
+    } finally {
+      out.close();
+    }
+    return true;
+  }
+  
+  private void writeRecursively(DataOutputStream out, TSTNode node) throws IOException {
+    if (node == null) {
+      return;
+    }
+    out.writeChar(node.splitchar);
+    byte mask = 0;
+    if (node.relatives[TSTNode.LOKID] != null) mask |= LO_KID;
+    if (node.relatives[TSTNode.EQKID] != null) mask |= EQ_KID;
+    if (node.relatives[TSTNode.HIKID] != null) mask |= HI_KID;
+    if (node.data != null) mask |= HAS_VALUE;
+    out.writeByte(mask);
+    if (node.data != null) {
+      out.writeFloat((Float)node.data);
+    }
+    writeRecursively(out, node.relatives[TSTNode.LOKID]);
+    writeRecursively(out, node.relatives[TSTNode.EQKID]);
+    writeRecursively(out, node.relatives[TSTNode.HIKID]);
   }
-
 }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/jaspell/JaspellTernarySearchTrie.java Mon Mar 28 10:50:28 2011
@@ -1,5 +1,34 @@
 package org.apache.solr.spelling.suggest.jaspell;
 
+/** 
+ * Copyright (c) 2005 Bruno Martins
+ * All rights reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without 
+ * modification, are permitted provided that the following conditions 
+ * are met:
+ * 1. Redistributions of source code must retain the above copyright 
+ *    notice, this list of conditions and the following disclaimer.
+ * 2. Redistributions in binary form must reproduce the above copyright
+ *    notice, this list of conditions and the following disclaimer in the
+ *    documentation and/or other materials provided with the distribution.
+ * 3. Neither the name of the organization nor the names of its contributors
+ *    may be used to endorse or promote products derived from this software
+ *    without specific prior written permission.
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE 
+ * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF
+ * THE POSSIBILITY OF SUCH DAMAGE.
+ */
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileInputStream;
@@ -127,6 +156,16 @@ public class JaspellTernarySearchTrie {
    */
   public JaspellTernarySearchTrie() {
   }
+  
+  // for loading
+  void setRoot(TSTNode newRoot) {
+    rootNode = newRoot;
+  }
+  
+  // for saving
+  TSTNode getRoot() {
+    return rootNode;
+  }
 
   /**
    * Constructs a Ternary Search Trie and loads data from a <code>File</code>

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/spelling/suggest/tst/TSTLookup.java Mon Mar 28 10:50:28 2011
@@ -1,6 +1,10 @@
 package org.apache.solr.spelling.suggest.tst;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -13,8 +17,8 @@ import org.apache.solr.util.SortedIterat
 import org.apache.solr.util.TermFreqIterator;
 
 public class TSTLookup extends Lookup {
-  TernaryTreeNode root;
-  TSTAutocomplete autocomplete;
+  TernaryTreeNode root = new TernaryTreeNode();
+  TSTAutocomplete autocomplete = new TSTAutocomplete();
 
   @Override
   public void init(NamedList config, SolrCore core) {
@@ -23,7 +27,6 @@ public class TSTLookup extends Lookup {
   @Override
   public void build(TermFreqIterator tfit) throws IOException {
     root = new TernaryTreeNode();
-    autocomplete = new TSTAutocomplete();
     // buffer first
     if (!(tfit instanceof SortedIterator)) {
       // make sure it's sorted
@@ -48,7 +51,16 @@ public class TSTLookup extends Lookup {
 
   @Override
   public Object get(String key) {
-    throw new UnsupportedOperationException("get() is not supported here");
+    List<TernaryTreeNode> list = autocomplete.prefixCompletion(root, key, 0);
+    if (list == null || list.isEmpty()) {
+      return null;
+    }
+    for (TernaryTreeNode n : list) {
+      if (n.token.equals(key)) {
+        return n.val;
+      }
+    }
+    return null;
   }
 
   @Override
@@ -75,15 +87,94 @@ public class TSTLookup extends Lookup {
     }
     return res;
   }
+  
+  public static final String FILENAME = "tst.dat";
+  
+  private static final byte LO_KID = 0x01;
+  private static final byte EQ_KID = 0x02;
+  private static final byte HI_KID = 0x04;
+  private static final byte HAS_TOKEN = 0x08;
+  private static final byte HAS_VALUE = 0x10;
 
   @Override
-  public boolean load(File storeDir) throws IOException {
-    return false;
+  public synchronized boolean load(File storeDir) throws IOException {
+    File data = new File(storeDir, FILENAME);
+    if (!data.exists() || !data.canRead()) {
+      return false;
+    }
+    DataInputStream in = new DataInputStream(new FileInputStream(data));
+    root = new TernaryTreeNode();
+    try {
+      readRecursively(in, root);
+    } finally {
+      in.close();
+    }
+    return true;
+  }
+  
+  // pre-order traversal
+  private void readRecursively(DataInputStream in, TernaryTreeNode node) throws IOException {
+    node.splitchar = in.readChar();
+    byte mask = in.readByte();
+    if ((mask & HAS_TOKEN) != 0) {
+      node.token = in.readUTF();
+    }
+    if ((mask & HAS_VALUE) != 0) {
+      node.val = new Float(in.readFloat());
+    }
+    if ((mask & LO_KID) != 0) {
+      node.loKid = new TernaryTreeNode();
+      readRecursively(in, node.loKid);
+    }
+    if ((mask & EQ_KID) != 0) {
+      node.eqKid = new TernaryTreeNode();
+      readRecursively(in, node.eqKid);
+    }
+    if ((mask & HI_KID) != 0) {
+      node.hiKid = new TernaryTreeNode();
+      readRecursively(in, node.hiKid);
+    }
   }
 
   @Override
-  public boolean store(File storeDir) throws IOException {
-    return false;
+  public synchronized boolean store(File storeDir) throws IOException {
+    if (!storeDir.exists() || !storeDir.isDirectory() || !storeDir.canWrite()) {
+      return false;
+    }
+    File data = new File(storeDir, FILENAME);
+    DataOutputStream out = new DataOutputStream(new FileOutputStream(data));
+    try {
+      writeRecursively(out, root);
+      out.flush();
+    } finally {
+      out.close();
+    }
+    return true;
+  }
+  
+  // pre-order traversal
+  private void writeRecursively(DataOutputStream out, TernaryTreeNode node) throws IOException {
+    // write out the current node
+    out.writeChar(node.splitchar);
+    // prepare a mask of kids
+    byte mask = 0;
+    if (node.eqKid != null) mask |= EQ_KID;
+    if (node.loKid != null) mask |= LO_KID;
+    if (node.hiKid != null) mask |= HI_KID;
+    if (node.token != null) mask |= HAS_TOKEN;
+    if (node.val != null) mask |= HAS_VALUE;
+    out.writeByte(mask);
+    if (node.token != null) out.writeUTF(node.token);
+    if (node.val != null) out.writeFloat((Float)node.val);
+    // recurse and write kids
+    if (node.loKid != null) {
+      writeRecursively(out, node.loKid);
+    }
+    if (node.eqKid != null) {
+      writeRecursively(out, node.eqKid);
+    }
+    if (node.hiKid != null) {
+      writeRecursively(out, node.hiKid);
+    }
   }
-
 }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/update/DocumentBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/update/DocumentBuilder.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/update/DocumentBuilder.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/update/DocumentBuilder.java Mon Mar 28 10:50:28 2011
@@ -18,12 +18,10 @@
 package org.apache.solr.update;
 
 import java.util.ArrayList;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrException;
@@ -194,7 +192,7 @@ public class DocumentBuilder {
   }
 
 
-  private static void addField(Document doc, SchemaField field, String val, float boost) {
+  private static void addField(Document doc, SchemaField field, Object val, float boost) {
     if (field.isPolyField()) {
       Fieldable[] farr = field.getType().createFields(field, val, boost);
       for (Fieldable f : farr) {
@@ -206,6 +204,15 @@ public class DocumentBuilder {
     }
   }
   
+  private static String getID( SolrInputDocument doc, IndexSchema schema )
+  {
+    String id = "";
+    SchemaField sf = schema.getUniqueKeyField();
+    if( sf != null ) {
+      id = "[doc="+doc.getFieldValue( sf.getName() )+"] ";
+    }
+    return id;
+  }
 
   /**
    * Convert a SolrInputDocument to a lucene Document.
@@ -235,91 +242,70 @@ public class DocumentBuilder {
       
       // Make sure it has the correct number
       if( sfield!=null && !sfield.multiValued() && field.getValueCount() > 1 ) {
-        String id = "";
-        SchemaField sf = schema.getUniqueKeyField();
-        if( sf != null ) {
-          id = "["+doc.getFieldValue( sf.getName() )+"] ";
-        }
         throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
-            "ERROR: "+id+"multiple values encountered for non multiValued field " + 
+            "ERROR: "+getID(doc, schema)+"multiple values encountered for non multiValued field " + 
               sfield.getName() + ": " +field.getValue() );
       }
       
 
       // load each field value
       boolean hasField = false;
-      for( Object v : field ) {
-        if( v == null ) {
-          continue;
-        }
-        String val = null;
-        hasField = true;
-        boolean isBinaryField = false;
-        if (sfield != null && sfield.getType() instanceof BinaryField) {
-          isBinaryField = true;
-          BinaryField binaryField = (BinaryField) sfield.getType();
-          Fieldable f = binaryField.createField(sfield,v,boost);
-          if(f != null){
-            out.add(f);
+      try {
+        for( Object v : field ) {
+          if( v == null ) {
+            continue;
           }
-          used = true;
-        } else {
-          // TODO!!! HACK -- date conversion
-          if (sfield != null && v instanceof Date && sfield.getType() instanceof DateField) {
-            DateField df = (DateField) sfield.getType();
-            val = df.toInternal((Date) v) + 'Z';
-          } else if (v != null) {
-            val = v.toString();
-          }
-
+          hasField = true;
           if (sfield != null) {
             used = true;
-            addField(out, sfield, val, boost);
-          }
-        }
-
-        // Check if we should copy this field to any other fields.
-        // This could happen whether it is explicit or not.
-        List<CopyField> copyFields = schema.getCopyFieldsList(name);
-        for (CopyField cf : copyFields) {
-          SchemaField destinationField = cf.getDestination();
-          // check if the copy field is a multivalued or not
-          if (!destinationField.multiValued() && out.get(destinationField.getName()) != null) {
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                    "ERROR: multiple values encountered for non multiValued copy field " +
-                            destinationField.getName() + ": " + val);
+            addField(out, sfield, v, boost);
           }
-
-          used = true;
-          //Don't worry about poly fields here
-          Fieldable [] fields = null;
-          if (isBinaryField) {
-            if (destinationField.getType() instanceof BinaryField) {
-              BinaryField binaryField = (BinaryField) destinationField.getType();
-              //TODO: safe to assume that binary fields only create one?
-              fields = new Fieldable[]{binaryField.createField(destinationField, v, boost)};
+  
+          // Check if we should copy this field to any other fields.
+          // This could happen whether it is explicit or not.
+          List<CopyField> copyFields = schema.getCopyFieldsList(name);
+          for (CopyField cf : copyFields) {
+            SchemaField destinationField = cf.getDestination();
+            // check if the copy field is a multivalued or not
+            if (!destinationField.multiValued() && out.getFieldable(destinationField.getName()) != null) {
+              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+                      "ERROR: "+getID(doc, schema)+"multiple values encountered for non multiValued copy field " +
+                              destinationField.getName() + ": " + v);
             }
-          } else {
-            fields = destinationField.createFields(cf.getLimitedValue(val), boost);
-          }
-          if (fields != null) { // null fields are not added
-            for (Fieldable f : fields) {
-              out.add(f);
+  
+            used = true;
+            
+            // Perhaps trim the length of a copy field
+            Object val = v;
+            if( val instanceof String && cf.getMaxChars() > 0 ) {
+              val = cf.getLimitedValue((String)val);
+            }
+            
+            Fieldable [] fields = destinationField.createFields(val, boost);
+            if (fields != null) { // null fields are not added
+              for (Fieldable f : fields) {
+                if(f != null) out.add(f);
+              }
             }
           }
+          
+          // In lucene, the boost for a given field is the product of the 
+          // document boost and *all* boosts on values of that field. 
+          // For multi-valued fields, we only want to set the boost on the
+          // first field.
+          boost = 1.0f; 
         }
-        
-        // In lucene, the boost for a given field is the product of the 
-        // document boost and *all* boosts on values of that field. 
-        // For multi-valued fields, we only want to set the boost on the
-        // first field.
-        boost = 1.0f; 
+      }
+      catch( Exception ex ) {
+        throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
+            "ERROR: "+getID(doc, schema)+"Error adding field '" + 
+              field.getName() + "'='" +field.getValue()+"'", ex );
       }
       
       // make sure the field was used somehow...
       if( !used && hasField ) {
-        throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,"ERROR:unknown field '" +
-                name + "'");
+        throw new SolrException( SolrException.ErrorCode.BAD_REQUEST,
+            "ERROR: "+getID(doc, schema)+"unknown field '" +name + "'");
       }
     }
     
@@ -332,8 +318,7 @@ public class DocumentBuilder {
           addField(out, field, field.getDefaultValue(), 1.0f);
         } 
         else {
-          String id = schema.printableUniqueKey( out );
-          String msg = "Document ["+id+"] missing required field: " + field.getName();
+          String msg = getID(doc, schema) + "missing required field: " + field.getName();
           throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, msg );
         }
       }

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/SolrPluginUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/SolrPluginUtils.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/SolrPluginUtils.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/SolrPluginUtils.java Mon Mar 28 10:50:28 2011
@@ -115,41 +115,6 @@ public class SolrPluginUtils {
      return splitList.split(value.trim(), 0);
   }
 
-  /**
-   * Assumes the standard query param of "fl" to specify the return fields
-   * @see #setReturnFields(String,SolrQueryResponse)
-   */
-  public static int setReturnFields(SolrQueryRequest req,
-                                    SolrQueryResponse res) {
-
-    return setReturnFields(req.getParams().get(CommonParams.FL), res);
-  }
-
-  /**
-   * Given a space seperated list of field names, sets the field list on the
-   * SolrQueryResponse.
-   *
-   * @return bitfield of SolrIndexSearcher flags that need to be set
-   */
-  public static int setReturnFields(String fl,
-                                    SolrQueryResponse res) {
-    int flags = 0;
-    if (fl != null) {
-      // TODO - this could become more efficient if widely used.
-      // TODO - should field order be maintained?
-      String[] flst = split(fl);
-      if (flst.length > 0 && !(flst.length==1 && flst[0].length()==0)) {
-        Set<String> set = new LinkedHashSet<String>();
-        for (String fname : flst) {
-          if("score".equalsIgnoreCase(fname))
-            flags |= SolrIndexSearcher.GET_SCORES;
-          set.add(fname);
-        }
-        res.setReturnFields(set);
-      }
-    }
-    return flags;
-  }
 
   /**
    * Pre-fetch documents into the index searcher's document cache.
@@ -180,14 +145,13 @@ public class SolrPluginUtils {
       return;
     }
 
-    Set<String> returnFields = res.getReturnFields();
-    Set<String> fieldFilter = returnFields;
-
-    if(returnFields != null) {
+    ReturnFields returnFields = res.getReturnFields();
+    if(returnFields.getLuceneFieldNames() != null) {
+      Set<String> fieldFilter = returnFields.getLuceneFieldNames();
 
       if (rb.doHighlights) {
         // copy return fields list
-        fieldFilter = new HashSet<String>(returnFields);
+        fieldFilter = new HashSet<String>(fieldFilter);
         // add highlight fields
 
         SolrHighlighter highlighter = HighlightComponent.getHighlighter(req.getCore());

Modified: lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java (original)
+++ lucene/dev/branches/docvalues/solr/src/java/org/apache/solr/util/xslt/TransformerProvider.java Mon Mar 28 10:50:28 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.io.IOUtils;
 
 import javax.xml.transform.Templates;
 import javax.xml.transform.Transformer;
@@ -29,6 +30,8 @@ import javax.xml.transform.TransformerFa
 import javax.xml.transform.stream.StreamSource;
 
 import org.apache.solr.common.ResourceLoader;
+import org.apache.solr.common.util.SystemIdResolver;
+import org.apache.solr.common.util.XMLErrorLogger;
 import org.apache.solr.core.SolrConfig;
 
 /** Singleton that creates a Transformer for the XSLTServletFilter.
@@ -40,19 +43,17 @@ import org.apache.solr.core.SolrConfig;
  */
 
 public class TransformerProvider {
-  public static TransformerProvider instance = new TransformerProvider();
-
-  private final TransformerFactory tFactory = TransformerFactory.newInstance();
   private String lastFilename;
   private Templates lastTemplates = null;
   private long cacheExpires = 0;
   
-  private static Logger log;
+  private static final Logger log = LoggerFactory.getLogger(TransformerProvider.class.getName());
+  private static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
   
+  public static TransformerProvider instance = new TransformerProvider();
+
   /** singleton */
   private TransformerProvider() {
-    log = LoggerFactory.getLogger(TransformerProvider.class.getName());
-    
     // tell'em: currently, we only cache the last used XSLT transform, and blindly recompile it
     // once cacheLifetimeSeconds expires
     log.warn(
@@ -99,8 +100,18 @@ public class TransformerProvider {
       if(log.isDebugEnabled()) {
         log.debug("compiling XSLT templates:" + filename);
       }
-      final InputStream xsltStream = loader.openResource("xslt/" + filename);
-      result = tFactory.newTemplates(new StreamSource(xsltStream));
+      final String fn = "xslt/" + filename;
+      final TransformerFactory tFactory = TransformerFactory.newInstance();
+      tFactory.setURIResolver(new SystemIdResolver(loader).asURIResolver());
+      tFactory.setErrorListener(xmllog);
+      final StreamSource src = new StreamSource(loader.openResource(fn),
+        SystemIdResolver.createSystemIdFromResourceName(fn));
+      try {
+        result = tFactory.newTemplates(src);
+      } finally {
+        // some XML parsers are broken and don't close the byte stream (but they should according to spec)
+        IOUtils.closeQuietly(src.getInputStream());
+      }
     } catch (Exception e) {
       log.error(getClass().getName(), "newTemplates", e);
       final IOException ioe = new IOException("Unable to initialize Templates '" + filename + "'");

Modified: lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/features.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/features.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/features.xml (original)
+++ lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/features.xml Mon Mar 28 10:50:28 2011
@@ -28,7 +28,7 @@
 <section>
 <title>Solr in a Nutshell</title>
   <p>
-  Solr is a standalone enterprise search server with a web-services like API. You put documents in it (called "indexing") via XML over HTTP. You query it via HTTP GET and receive XML results.
+  Solr is a standalone enterprise search server with a REST-like API. You put documents in it (called "indexing") via XML, JSON or binary over HTTP. You query it via HTTP GET and receive XML, JSON, or binary results.
   </p>
   <ul>
     <li> Advanced Full-Text Search Capabilities </li>
@@ -48,6 +48,7 @@
     <li> A Real Data Schema, with Numeric Types, Dynamic Fields, Unique Keys </li>
     <li> Powerful Extensions to the Lucene Query Language </li>
     <li> Faceted Search and Filtering </li>
+    <li> Geospatial Search </li>
     <li> Advanced, Configurable Text Analysis </li>
     <li> Highly Configurable and User Extensible Caching </li>
     <li> Performance Optimizations </li>
@@ -56,9 +57,10 @@
     <li> Monitorable Logging </li>
     <li> Fast Incremental Updates and Index Replication </li>
     <li> Highly Scalable Distributed search with sharded index across multiple hosts </li>
-    <li> XML, CSV/delimited-text, and binary update formats </li>
+    <li> JSON, XML, CSV/delimited-text, and binary update formats </li>
     <li> Easy ways to pull in data from databases and XML files from local disk and HTTP sources </li>
     <li> Rich Document Parsing and Indexing (PDF, Word, HTML, etc) using Apache Tika </li>
+    <li> Apache UIMA integration for configurable metadata extraction </li>
     <li> Multiple search indices </li>
   </ul>
 </section>
@@ -82,10 +84,10 @@
   <section><title>Query</title>
     <ul>
       <li>HTTP interface with configurable response formats (XML/XSLT, JSON, Python, Ruby, PHP, Velocity, binary)</li>
-      <li>Sort by any number of fields</li>
+      <li>Sort by any number of fields, and by complex functions of numeric fields</li>
       <li>Advanced DisMax query parser for high relevancy results from user-entered queries</li> 
       <li>Highlighted context snippets</li>
-      <li>Faceted Searching based on unique field values, explicit queries, or date ranges</li>
+      <li>Faceted Searching based on unique field values, explicit queries, date ranges, and numeric ranges</li>
       <li>Multi-Select Faceting by tagging and selectively excluding filters</li>
       <li>Spelling suggestions for user queries</li>
       <li>More Like This suggestions for given document</li>
@@ -96,7 +98,7 @@
       <li>Dynamic search results clustering using Carrot2</li>
       <li>Numeric field statistics such as min, max, average, standard deviation </li>
       <li>Combine queries derived from different syntaxes</li>
-      <li>Auto-suggest functionality</li>
+      <li>Auto-suggest functionality for completing user queries</li>
       <li>Allow configuration of top results for a query, overriding normal scoring and sorting</li>
       <li>Performance Optimizations</li>
     </ul>

Modified: lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/index.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/index.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/index.xml (original)
+++ lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/index.xml Mon Mar 28 10:50:28 2011
@@ -28,8 +28,8 @@
 Solr is the popular, blazing fast open source enterprise search
 platform from the Apache Lucene project.  Its major features include
 powerful full-text search, hit highlighting, faceted search, dynamic
-clustering, database integration, and rich document (e.g., Word, PDF)
-handling.  Solr is highly scalable, providing distributed search and
+clustering, database integration, rich document (e.g., Word, PDF)
+handling, and geospatial search.  Solr is highly scalable, providing distributed search and
 index replication, and it powers the search and navigation features of
 many of the world's largest internet sites.
 </p>
@@ -66,6 +66,32 @@ customization is required.
     </section>
     <section id="news">
       <title>News</title>
+            <section>
+   <title>March 2011 - Solr 3.1 Released</title>
+   <p>
+     Solr 3.1 has been released and is now available for public <a href="http://www.apache.org/dyn/closer.cgi/lucene/solr/">download</a>!
+     
+   New Solr 3.1 features include
+ </p>
+  <ul>
+    <li>Improved geospatial support</li>
+    <li>Sorting by function queries</li>
+    <li>Range faceting on all numeric fields</li>
+    <li>Example Velocity driven search UI at http://localhost:8983/solr/browse</li>
+    <li>A new termvector-based highlighter</li>
+    <li>Improved spellchecking capabilities</li>
+    <li>Improved integration with Apache Lucene</li>
+    <li>New autosuggest component</li>
+    <li>Distributed support for more components</li>
+    <li>JSON document indexing and CSV response format</li>
+    <li>Apache UIMA integration for metadata extraction</li>
+    <li>Many other Bugfixes, improvements and optimizations</li>
+  </ul>
+     
+	<p>See the <a href="http://svn.apache.org/repos/asf/lucene/dev/tags/lucene_solr_3_1/solr/CHANGES.txt">release notes</a> for more details.
+ </p>
+
+</section>
         <section>
           <title>25 June 2010 - Solr 1.4.1 Released</title>
           <p>

Modified: lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/tutorial.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/tutorial.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/tutorial.xml (original)
+++ lucene/dev/branches/docvalues/solr/src/site/src/documentation/content/xdocs/tutorial.xml Mon Mar 28 10:50:28 2011
@@ -46,10 +46,9 @@ To follow along with this tutorial, you 
 
 <ol>
   <li>Java 1.6 or greater.  Some places you can get it are from
-  <a href="http://java.sun.com/j2se/downloads.html">OpenJDK</a>,
-  <a href="http://java.sun.com/j2se/downloads.html">Sun</a>,
+  <a href="http://www.oracle.com/technetwork/java/javase/downloads/index.html">Oracle</a>,
+  <a href="http://openjdk.java.net/">Open JDK</a>,
   <a href="http://www.ibm.com/developerworks/java/jdk/">IBM</a>, or
-  <a href="http://www.oracle.com/technology/products/jrockit/index.html">Oracle</a>.
   <br/>
   Running <code>java -version</code> at the command line should indicate a version
   number starting with 1.6.  Gnu's GCJ is not supported and does not work with Solr.
@@ -86,7 +85,7 @@ To launch Jetty with the Solr WAR, and t
 <source>
 user:~/solr/example$ <strong>java -jar start.jar</strong>
 2009-10-23 16:42:53.816::INFO:  Logging to STDERR via org.mortbay.log.StdErrLog
-2009-10-23 16:42:53.907::INFO:  jetty-6.1.3
+2009-10-23 16:42:53.907::INFO:  jetty-6.1.26
 
 ...
 
@@ -141,11 +140,11 @@ You can now search for "solr" using the 
 Clicking the "Search" button should take you to the following URL...
 </p>
 <p>
-<a href="http://localhost:8983/solr/select/?q=solr&amp;version=2.2&amp;start=0&amp;rows=10&amp;indent=on">http://localhost:8983/solr/select/?q=solr&amp;version=2.2&amp;start=0&amp;rows=10&amp;indent=on</a>
+<a href="http://localhost:8983/solr/select/?q=solr&amp;start=0&amp;rows=10&amp;indent=on">http://localhost:8983/solr/select/?q=solr&amp;start=0&amp;rows=10&amp;indent=on</a>
 </p>
 
 <p>
-You can index all of the sample data, using the following command (assuming your shell supports the *.xml notation):
+You can index all of the sample data, using the following command (assuming your command line shell supports the *.xml notation):
 </p>
 
 <source>
@@ -189,6 +188,8 @@ SimplePostTool: COMMITting Solr index ch
   <li><a href="http://wiki.apache.org/solr/UpdateCSV">Load a CSV file</a> (comma separated values),
    including those exported by Excel or MySQL.
   </li> 
+  <li><a href="http://wiki.apache.org/solr/UpdateJSON">POST JSON documents</a>
+  </li> 
   <li>Index binary documents such as Word and PDF with 
     <a href="http://wiki.apache.org/solr/ExtractingRequestHandler">Solr Cell</a> (ExtractingRequestHandler).
   </li>
@@ -313,6 +314,14 @@ in subsequent searches.
     </ul>
 
     <p>
+      Complex functions may also be used to sort results...
+    </p>
+    <ul>
+      <li><a href="http://localhost:8983/solr/select/?indent=on&amp;q=*:*&amp;sort=div(popularity,add(price,1))+desc">q=video&amp;sort=div(popularity,add(price,1)) desc</a></li>
+    </ul>
+
+
+    <p>
       If no sort is specified, the default is <code>score desc</code> to return the matches having the highest relevancy.
     </p>
 
@@ -402,6 +411,21 @@ in subsequent searches.
 
 
 <section>
+  <title>Search UI</title>
+  <p>
+    Solr includes an example search interface built with velocity templating
+    that demonstrates many features, including searching, faceting, highlighting,
+    autocomplete, and geospatial searching. 
+  </p>
+  <p>
+    Try it out at
+    <a href="http://localhost:8983/solr/browse">http://localhost:8983/solr/browse</a>
+  </p>
+</section> <!-- Search UI -->
+
+
+
+<section>
   <title>Text Analysis</title>
 
   <p>

Modified: lucene/dev/branches/docvalues/solr/src/site/src/documentation/resources/schema/catalog.xcat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/site/src/documentation/resources/schema/catalog.xcat?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/site/src/documentation/resources/schema/catalog.xcat (original)
+++ lucene/dev/branches/docvalues/solr/src/site/src/documentation/resources/schema/catalog.xcat Mon Mar 28 10:50:28 2011
@@ -21,6 +21,5 @@
 <catalog xmlns="urn:oasis:names:tc:entity:xmlns:xml:catalog"
          prefer="public">
   <!-- Sets of symbols. e.g. for string replacements -->
-  <public publicId="-//Apache Solr//ENTITIES Symbols Project v1.0//EN"
-        uri="../../../../../../build/solr-specific-forrest-variables.ent"/>
+
 </catalog>

Modified: lucene/dev/branches/docvalues/solr/src/site/src/documentation/skinconf.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/site/src/documentation/skinconf.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/site/src/documentation/skinconf.xml (original)
+++ lucene/dev/branches/docvalues/solr/src/site/src/documentation/skinconf.xml Mon Mar 28 10:50:28 2011
@@ -21,12 +21,7 @@ Skin configuration file. This file conta
 which will be used to configure the chosen Forrest skin.
 -->
 
-<!DOCTYPE skinconfig PUBLIC "-//APACHE//DTD Skin Configuration V0.7-1//EN" "http://forrest.apache.org/dtd/skinconfig-v07-1.dtd" [
-<!ENTITY % symbols-project
-    PUBLIC "-//Apache Solr//ENTITIES Symbols Project v1.0//EN"
-    "solr-specific-forrest-variables.ent"> 
-  %symbols-project;
-]>
+<!DOCTYPE skinconfig PUBLIC "-//APACHE//DTD Skin Configuration V0.7-1//EN" "http://forrest.apache.org/dtd/skinconfig-v07-1.dtd">
 <skinconfig>
   <!-- To enable lucene search add provider="lucene" (default is google).
     Add box-location="alt" to move the search box to an alternate location
@@ -156,16 +151,6 @@ which will be used to configure the chos
       both : both
     -->
 
-  <motd>
-    <motd-option pattern="tutorial">
-      <motd-title>version &solr.specversion;</motd-title>
-      <motd-page location="both">
-      This document is for Apache Solr version &solr.specversion;.  If you are using a different version of Solr, please consult the documentation that was distributed with the version you are using.
-        </motd-page>
-    </motd-option>
-  </motd>
-
-
   <!--
     extra-css - here you can define custom css-elements that are 
     A) overriding the fallback elements or 

Modified: lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/impl/XMLResponseParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/impl/XMLResponseParser.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/impl/XMLResponseParser.java (original)
+++ lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/impl/XMLResponseParser.java Mon Mar 28 10:50:28 2011
@@ -37,8 +37,10 @@ import org.apache.solr.client.solrj.util
 import org.apache.solr.common.SolrDocument;
 import org.apache.solr.common.SolrDocumentList;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.DateUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.XMLErrorLogger;
 
 /**
  * 
@@ -48,6 +50,7 @@ import org.apache.solr.common.util.Simpl
 public class XMLResponseParser extends ResponseParser
 {
   public static Logger log = LoggerFactory.getLogger(XMLResponseParser.class);
+  private static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
 
   // reuse the factory among all parser instances so things like string caches
   // won't be duplicated
@@ -68,6 +71,7 @@ public class XMLResponseParser extends R
       // isimplementation specific.
       log.debug( "Unable to set the 'reuse-instance' property for the input factory: "+factory );
     }
+    factory.setXMLReporter(xmllog);
   }
 
   public XMLResponseParser() {}
@@ -162,7 +166,7 @@ public class XMLResponseParser extends R
       @Override 
       public Date read( String txt ) { 
         try {
-          return ClientUtils.parseDate(txt);      
+          return DateUtil.parseDate(txt);      
         }
         catch( Exception ex ) {
           ex.printStackTrace();

Modified: lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/util/ClientUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/util/ClientUtils.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/util/ClientUtils.java (original)
+++ lucene/dev/branches/docvalues/solr/src/solrj/org/apache/solr/client/solrj/util/ClientUtils.java Mon Mar 28 10:50:28 2011
@@ -48,7 +48,7 @@ import org.apache.solr.common.util.*;
 public class ClientUtils 
 {
   // Standard Content types
-  public static final String TEXT_XML = "text/xml; charset=utf-8";  
+  public static final String TEXT_XML = "application/xml; charset=UTF-8";  
   
   /**
    * Take a string and make it an iterable ContentStream

Modified: lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-spellchecker.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-spellchecker.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-spellchecker.xml (original)
+++ lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-spellchecker.xml Mon Mar 28 10:50:28 2011
@@ -37,6 +37,7 @@
       <str name="classname">org.apache.solr.spelling.suggest.Suggester</str>
       <str name="lookupImpl">org.apache.solr.spelling.suggest.jaspell.JaspellLookup</str>
       <str name="field">suggest</str>
+      <str name="storeDir">suggest</str>
       <str name="buildOnCommit">true</str>
 
       <!-- Suggester properties -->

Modified: lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-xinclude.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-xinclude.xml?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-xinclude.xml (original)
+++ lucene/dev/branches/docvalues/solr/src/test-files/solr/conf/solrconfig-xinclude.xml Mon Mar 28 10:50:28 2011
@@ -19,6 +19,10 @@
 
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
-  <!--  FIXME: is there a way to make this test *not* depend on the CWD? -->
-  <xi:include href="solrconfig-reqHandler.incl" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+
+  <xi:include href="foobar-missing.xml" xmlns:xi="http://www.w3.org/2001/XInclude">
+    <xi:fallback>
+      <xi:include href="solrconfig-reqHandler.incl" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+    </xi:fallback>
+  </xi:include>
 </config>

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/BasicFunctionalityTest.java Mon Mar 28 10:50:28 2011
@@ -42,6 +42,7 @@ import org.apache.solr.handler.RequestHa
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
+import org.apache.solr.response.ResultContext;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.response.XMLWriter;
 import org.apache.solr.schema.IndexSchema;
@@ -100,7 +101,7 @@ public class BasicFunctionalityTest exte
   
   @Test
   public void testIgnoredFields() throws Exception {
-    lrf.args.put("version","2.0");
+    lrf.args.put(CommonParams.VERSION,"2.2");
     assertU("adding doc with ignored field",
             adoc("id", "42", "foo_ignored", "blah blah"));
     assertU("commit",
@@ -123,7 +124,7 @@ public class BasicFunctionalityTest exte
     assertEquals("Mergefactor was not picked up", ((LogMergePolicy) writer.getConfig().getMergePolicy()).getMergeFactor(), 8);
     writer.close();
 
-    lrf.args.put("version","2.0");
+    lrf.args.put(CommonParams.VERSION,"2.2");
     assertQ("test query on empty index",
             req("qlkciyopsbgzyvkylsjhchghjrdf")
             ,"//result[@numFound='0']"
@@ -445,7 +446,7 @@ public class BasicFunctionalityTest exte
   @Test
   public void testDefaultFieldValues() {
     clearIndex();
-    lrf.args.put("version","2.1");
+    lrf.args.put(CommonParams.VERSION,"2.2");
     assertU(adoc("id",  "4055",
                  "subject", "Hoss the Hoss man Hostetter"));
     assertU(adoc("id",  "4056",
@@ -558,7 +559,7 @@ public class BasicFunctionalityTest exte
     SolrQueryResponse rsp = new SolrQueryResponse();
     core.execute(core.getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
 
-    DocList dl = (DocList) rsp.getValues().get("response");
+    DocList dl = ((ResultContext) rsp.getValues().get("response")).docs;
     org.apache.lucene.document.Document d = req.getSearcher().doc(dl.iterator().nextDoc());
     // ensure field is not lazy
     assertTrue( d.getFieldable("test_hlt") instanceof Field );
@@ -580,7 +581,7 @@ public class BasicFunctionalityTest exte
     SolrQueryResponse rsp = new SolrQueryResponse();
     core.execute(core.getRequestHandler(req.getParams().get(CommonParams.QT)), req, rsp);
 
-    DocList dl = (DocList) rsp.getValues().get("response");
+    DocList dl = ((ResultContext) rsp.getValues().get("response")).docs;
     DocIterator di = dl.iterator();    
     org.apache.lucene.document.Document d = req.getSearcher().doc(di.nextDoc());
     // ensure field is lazy
@@ -675,6 +676,7 @@ public class BasicFunctionalityTest exte
     assertU(commit());
   
     try {
+      ignoreException("can not sort on multivalued field: sortabuse_t");
       assertQ("sort on something that shouldn't work",
               req("q", "sortabuse_b:true",
                   "sort", "sortabuse_t asc"),

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/ConvertedLegacyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/ConvertedLegacyTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/ConvertedLegacyTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/ConvertedLegacyTest.java Mon Mar 28 10:50:28 2011
@@ -17,6 +17,7 @@
 
 package org.apache.solr;
 
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.request.*;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -42,7 +43,7 @@ public class ConvertedLegacyTest extends
     // these may be reused by things that need a special query
     SolrQueryRequest req = null;
     Map<String,String> args = new HashMap<String,String>();
-    lrf.args.put("version","2.0");
+    lrf.args.put(CommonParams.VERSION,"2.2");
     lrf.args.put("defType","lucenePlusSort");
 
     // compact the index, keep things from getting out of hand
@@ -133,7 +134,6 @@ public class ConvertedLegacyTest extends
             ,"//*[@start='0']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 2, 5 , args);
     assertQ(req
@@ -143,7 +143,6 @@ public class ConvertedLegacyTest extends
             ,"//*[@start='2']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 3, 5 , args);
     assertQ(req
@@ -151,7 +150,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 4, 5 , args);
     assertQ(req
@@ -159,7 +157,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 25, 5 , args);
     assertQ(req
@@ -167,7 +164,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 0, 1 , args);
     assertQ(req
@@ -176,7 +172,6 @@ public class ConvertedLegacyTest extends
             ,"*//doc[1]/str[.='apple']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 0, 2 , args);
     assertQ(req
@@ -185,7 +180,6 @@ public class ConvertedLegacyTest extends
             ,"*//doc[2]/str[.='banana']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 1, 1 , args);
     assertQ(req
@@ -194,7 +188,6 @@ public class ConvertedLegacyTest extends
             ,"*//doc[1]/str[.='banana']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 3, 1 , args);
     assertQ(req
@@ -202,7 +195,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 4, 1 , args);
     assertQ(req
@@ -210,7 +202,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 1, 0 , args);
     assertQ(req
@@ -218,7 +209,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z]",
                                     "standard", 0, 0 , args);
     assertQ(req
@@ -226,7 +216,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("defType","lucenePlusSort");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z];val_s1 asc",
                                     "standard", 0, 0 , args);
@@ -235,7 +224,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc)=0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("defType","lucenePlusSort");
     req = new LocalSolrQueryRequest(h.getCore(), "val_s:[a TO z];val_s1 desc",
                                     "standard", 0, 0 , args);
@@ -480,13 +468,13 @@ public class ConvertedLegacyTest extends
 
     assertQ(req("id:44")
             ,"*[count(//doc/*)>=3]  "
-            ,"//int[@name='gack_i'][.='51778']  "
-            ,"//str[@name='t_name'][.='cats']"
+            ,"//arr[@name='gack_i']/int[.='51778']  "
+            ,"//arr[@name='t_name']/str[.='cats']"
             );
     // now test if we can query by a dynamic field (requires analyzer support)
 
     assertQ(req("t_name:cat")
-            ,"//str[@name='t_name' and .='cats']"
+            ,"//arr[@name='t_name' and .='cats']/str"
             );
     // check that deleteByQuery works for dynamic fields
 
@@ -501,8 +489,8 @@ public class ConvertedLegacyTest extends
     assertU("<add><doc><field name=\"id\">44</field><field name=\"xaa\">mystr</field><field name=\"xaaa\">12321</field></doc></add>");
     assertU("<commit/>");
     assertQ(req("id:44")
-            ,"//str[@name='xaa'][.='mystr']  "
-            ,"//int[@name='xaaa'][.='12321']"
+            ,"//arr[@name='xaa'][.='mystr']/str  "
+            ,"//arr[@name='xaaa'][.='12321']/int"
             );
 
 
@@ -761,28 +749,28 @@ public class ConvertedLegacyTest extends
             );
     assertQ(req("id_i:[1000 TO 1010]; a_i1 asc,b_si desc")
             ,"*[count(//doc)=7] "
-            ,"//doc[3]/int[.='100'] "
-            ,"//doc[4]/int[.='50']  "
-            ,"//doc[5]/int[.='1000']"
+            ,"//doc[3]/int[@name='b_si' and .='100'] "
+            ,"//doc[4]/int[@name='b_si' and .='50']  "
+            ,"//doc[5]/arr[@name='id_i' and .='1000']"
             );
     assertQ(req("id_i:[1000 TO 1010]; a_i1 asc,b_si asc")
             ,"*[count(//doc)=7] "
-            ,"//doc[3]/int[.='50'] "
-            ,"//doc[4]/int[.='100']  "
-            ,"//doc[5]/int[.='1000']"
+            ,"//doc[3]/int[@name='b_si' and .='50'] "
+            ,"//doc[4]/int[@name='b_si' and .='100']  "
+            ,"//doc[5]/arr[@name='id_i' and .='1000']"
             );
     // nullfirst tests
     assertQ(req("id_i:[1000 TO 1002]; nullfirst asc")
             ,"*[count(//doc)=3] "
-            ,"//doc[1]/int[.='1002']"
-            ,"//doc[2]/int[.='1001']  "
-            ,"//doc[3]/int[.='1000']"
+            ,"//doc[1]/arr[@name='id_i' and .='1002']"
+            ,"//doc[2]/arr[@name='id_i' and .='1001']  "
+            ,"//doc[3]/arr[@name='id_i' and .='1000']"
             );
     assertQ(req("id_i:[1000 TO 1002]; nullfirst desc")
             ,"*[count(//doc)=3] "
-            ,"//doc[1]/int[.='1002']"
-            ,"//doc[2]/int[.='1000']  "
-            ,"//doc[3]/int[.='1001']"
+            ,"//doc[1]/arr[@name='id_i' and .='1002']"
+            ,"//doc[2]/arr[@name='id_i' and .='1000']  "
+            ,"//doc[3]/arr[@name='id_i' and .='1001']"
             );
 
     ignoreException("shouldbeunindexed");
@@ -1126,7 +1114,6 @@ public class ConvertedLegacyTest extends
     assertQ(req("id:44")
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","fname_s,arr_f  ");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44",
                                     "standard", 0, 10, args);
@@ -1135,19 +1122,17 @@ public class ConvertedLegacyTest extends
             ,"//float[.='1.4142135']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
-    args.put("fl","  ");
+    args.put("fl","fname_s,score");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44",
                                     "standard", 0, 10, args);
     assertQ(req
-            ,"//str[.='Yonik']  "
-            ,"//float[.='1.4142135']"
+            ,"//str[.='Yonik']"
+            ,"//float[.='2.9459102']"
             );
 
     // test addition of score field
 
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","score ");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44",
                                     "standard", 0, 10, args);
@@ -1158,7 +1143,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc/*)>=13]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","*,score ");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44",
                                     "standard", 0, 10, args);
@@ -1169,7 +1153,6 @@ public class ConvertedLegacyTest extends
             ,"*[count(//doc/*)>=13]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","* ");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44",
                                     "standard", 0, 10, args);
@@ -1182,7 +1165,6 @@ public class ConvertedLegacyTest extends
     // test maxScore
 
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","score ");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44",
                                     "standard", 0, 10, args);
@@ -1190,7 +1172,6 @@ public class ConvertedLegacyTest extends
             ,"//result[@maxScore>0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","score ");
     args.put("defType","lucenePlusSort");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44;id desc;",
@@ -1199,7 +1180,6 @@ public class ConvertedLegacyTest extends
             ,"//result[@maxScore>0]"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","score ");
     args.put("defType","lucenePlusSort");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44;",
@@ -1208,7 +1188,6 @@ public class ConvertedLegacyTest extends
             ,"//@maxScore = //doc/float[@name='score']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","score ");
     args.put("defType","lucenePlusSort");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44;id desc;",
@@ -1217,7 +1196,6 @@ public class ConvertedLegacyTest extends
             ,"//@maxScore = //doc/float[@name='score']"
             );
     args = new HashMap<String,String>();
-    args.put("version","2.0");
     args.put("fl","score");
     args.put("defType","lucenePlusSort");
     req = new LocalSolrQueryRequest(h.getCore(), "id:44;id desc;",

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/DisMaxRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/DisMaxRequestHandlerTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/DisMaxRequestHandlerTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/DisMaxRequestHandlerTest.java Mon Mar 28 10:50:28 2011
@@ -32,7 +32,7 @@ public class DisMaxRequestHandlerTest ex
     initCore("solrconfig.xml","schema.xml");
     lrf = h.getRequestFactory
       ("dismax", 0, 20,
-       "version","2.0",
+       CommonParams.VERSION,"2.2",
        "facet", "true",
        "facet.field","t_s"
        );
@@ -94,7 +94,7 @@ public class DisMaxRequestHandlerTest ex
     assertQ("multi qf",
             req("q", "cool"
                 ,"qt", qt
-                ,"version", "2.0"
+                ,CommonParams.VERSION, "2.2"
                 ,"qf", "subject"
                 ,"qf", "features_t"
                 )
@@ -104,7 +104,7 @@ public class DisMaxRequestHandlerTest ex
     assertQ("boost query",
             req("q", "cool stuff"
                 ,"qt", qt
-                ,"version", "2.0"
+                ,CommonParams.VERSION, "2.2"
                 ,"bq", "subject:hell^400"
                 )
             ,"//*[@numFound='3']"
@@ -116,7 +116,7 @@ public class DisMaxRequestHandlerTest ex
     assertQ("multi boost query",
             req("q", "cool stuff"
                 ,"qt", qt
-                ,"version", "2.0"
+                ,CommonParams.VERSION, "2.2"
                 ,"bq", "subject:hell^400"
                 ,"bq", "subject:cool^4"
                 , CommonParams.DEBUG_QUERY, "true"
@@ -176,7 +176,7 @@ public class DisMaxRequestHandlerTest ex
     Pattern p_bool = Pattern.compile("\\(subject:hell\\s*subject:cool\\)");
     String resp = h.query(req("q", "cool stuff"
                 ,"qt", "dismax"
-                ,"version", "2.0"
+                ,CommonParams.VERSION, "2.2"
                 ,"bq", "subject:hell OR subject:cool"
                 ,CommonParams.DEBUG_QUERY, "true"
                               ));
@@ -185,7 +185,7 @@ public class DisMaxRequestHandlerTest ex
 
     resp = h.query(req("q", "cool stuff"
                 ,"qt", "dismax"
-                ,"version", "2.0"
+                ,CommonParams.VERSION, "2.2"
                 ,"bq", "subject:hell OR subject:cool"
                 ,"bq",""
                 ,CommonParams.DEBUG_QUERY, "true"

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/EchoParamsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/EchoParamsTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/EchoParamsTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/EchoParamsTest.java Mon Mar 28 10:50:28 2011
@@ -17,6 +17,7 @@
 
 package org.apache.solr;
 
+import org.apache.solr.common.params.CommonParams;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -34,7 +35,7 @@ public class EchoParamsTest extends Solr
   @Test
   public void testDefaultEchoParams() {
     lrf.args.put("wt", "xml");
-    lrf.args.put("version", "2.2");    
+    lrf.args.put(CommonParams.VERSION, "2.2");    
     assertQ(req("foo"),HEADER_XPATH + "/int[@name='status']");
     assertQ(req("foo"),"not(//lst[@name='params'])");
   }
@@ -42,7 +43,7 @@ public class EchoParamsTest extends Solr
   @Test
   public void testDefaultEchoParamsDefaultVersion() {
     lrf.args.put("wt", "xml");
-    lrf.args.remove("version");    
+    lrf.args.remove(CommonParams.VERSION);    
     assertQ(req("foo"),HEADER_XPATH + "/int[@name='status']");
     assertQ(req("foo"),"not(//lst[@name='params'])");
   }
@@ -50,7 +51,7 @@ public class EchoParamsTest extends Solr
   @Test
   public void testExplicitEchoParams() {
     lrf.args.put("wt", "xml");
-    lrf.args.put("version", "2.2");
+    lrf.args.put(CommonParams.VERSION, "2.2");
     lrf.args.put("echoParams", "explicit");
     assertQ(req("foo"),HEADER_XPATH + "/int[@name='status']");
     assertQ(req("foo"),HEADER_XPATH + "/lst[@name='params']");
@@ -61,7 +62,7 @@ public class EchoParamsTest extends Solr
   public void testAllEchoParams() {
     lrf = h.getRequestFactory
       ("crazy_custom_qt", 0, 20,
-       "version","2.2",
+       CommonParams.VERSION,"2.2",
        "wt","xml",
        "echoParams", "all",
        "echoHandler","true"

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/MinimalSchemaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/MinimalSchemaTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/MinimalSchemaTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/MinimalSchemaTest.java Mon Mar 28 10:50:28 2011
@@ -17,6 +17,7 @@
 
 package org.apache.solr;
 
+import org.apache.solr.common.params.CommonParams;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -44,7 +45,7 @@ public class MinimalSchemaTest extends S
     assertNull("UniqueKey Field isn't null", 
                h.getCore().getSchema().getUniqueKeyField());
 
-    lrf.args.put("version","2.0");
+    lrf.args.put(CommonParams.VERSION,"2.2");
 
     assertNull("Simple assertion that adding a document works", h.validateUpdate(
             adoc("id",  "4055",

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/OutputWriterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/OutputWriterTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/OutputWriterTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/OutputWriterTest.java Mon Mar 28 10:50:28 2011
@@ -20,6 +20,7 @@ package org.apache.solr;
 import java.io.IOException;
 import java.io.Writer;
 
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.QueryResponseWriter;
@@ -42,25 +43,21 @@ public class OutputWriterTest extends So
     }
     
     
-    /** responseHeader has changed in SOLR-59, check old and new variants */
+    /** 
+     * responseHeader has changed in SOLR-59, check old and new variants,
+     * In SOLR-2413, we removed support for the deprecated versions
+     */
     @Test
     public void testSOLR59responseHeaderVersions() {
         // default version is 2.2, with "new" responseHeader
-        lrf.args.remove("version");
+        lrf.args.remove(CommonParams.VERSION);
         lrf.args.put("wt", "standard");
         assertQ(req("foo"), "/response/lst[@name='responseHeader']/int[@name='status'][.='0']");
         lrf.args.remove("wt");
         assertQ(req("foo"), "/response/lst[@name='responseHeader']/int[@name='QTime']");
         
-        // version=2.1 reverts to old responseHeader
-        lrf.args.put("version", "2.1");
-        lrf.args.put("wt", "standard");
-        assertQ(req("foo"), "/response/responseHeader/status[.='0']");
-        lrf.args.remove("wt");
-        assertQ(req("foo"), "/response/responseHeader/QTime");
-
         // and explicit 2.2 works as default  
-        lrf.args.put("version", "2.2");
+        //lrf.args.put("version", "2.2");
         lrf.args.put("wt", "standard");
         assertQ(req("foo"), "/response/lst[@name='responseHeader']/int[@name='status'][.='0']");
         lrf.args.remove("wt");
@@ -83,6 +80,15 @@ public class OutputWriterTest extends So
         assertTrue(out.contains("DUMMY"));
     }
     
+    @Test
+    public void testTrivialXsltWriterInclude() throws Exception {
+        lrf.args.put("wt", "xslt");
+        lrf.args.put("tr", "dummy-using-include.xsl");
+        String out = h.query(req("foo"));
+        // System.out.println(out);
+        assertTrue(out.contains("DUMMY"));
+    }
+    
     
     ////////////////////////////////////////////////////////////////////////////
     /** An output writer that doesn't do anything useful. */

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SampleTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SampleTest.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SampleTest.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SampleTest.java Mon Mar 28 10:50:28 2011
@@ -17,6 +17,7 @@
 
 package org.apache.solr;
 
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.request.*;
 import org.apache.solr.util.*;
 import org.junit.BeforeClass;
@@ -46,7 +47,7 @@ public class SampleTest extends SolrTest
    */
   @Test
   public void testSimple() {
-    lrf.args.put("version","2.0");
+    lrf.args.put(CommonParams.VERSION,"2.2");
     assertU("Simple assertion that adding a document works",
             adoc("id",  "4055",
                  "subject", "Hoss the Hoss man Hostetter"));
@@ -70,7 +71,7 @@ public class SampleTest extends SolrTest
    */
   @Test
   public void testAdvanced() throws Exception {
-    lrf.args.put("version","2.0");        
+    lrf.args.put(CommonParams.VERSION,"2.2");        
     assertU("less common case, a complex addition with options",
             add(doc("id", "4059",
                     "subject", "Who Me?"),
@@ -102,7 +103,7 @@ public class SampleTest extends SolrTest
      * Note: the qt proves we are using our custom config...
      */
     TestHarness.LocalRequestFactory l = h.getRequestFactory
-      ("crazy_custom_qt",100,200,"version","2.1");
+      ("crazy_custom_qt",100,200,CommonParams.VERSION,"2.2");
     assertQ("how did i find Mack Daddy? ",
             l.makeRequest( "Mack Daddy" )
             ,"//result[@numFound=0]"

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SolrTestCaseJ4.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SolrTestCaseJ4.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SolrTestCaseJ4.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/SolrTestCaseJ4.java Mon Mar 28 10:50:28 2011
@@ -26,6 +26,7 @@ import org.apache.noggit.ObjectBuilder;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.XML;
@@ -238,7 +239,7 @@ public abstract class SolrTestCaseJ4 ext
               solrConfig,
               getSchemaFile());
       lrf = h.getRequestFactory
-              ("standard",0,20,"version","2.2");
+              ("standard",0,20,CommonParams.VERSION,"2.2");
     }
     log.info("####initCore end");
   }

Modified: lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestDistributedSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestDistributedSearch.java?rev=1086181&r1=1086180&r2=1086181&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestDistributedSearch.java (original)
+++ lucene/dev/branches/docvalues/solr/src/test/org/apache/solr/TestDistributedSearch.java Mon Mar 28 10:50:28 2011
@@ -137,6 +137,7 @@ public class TestDistributedSearch exten
     query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2);
     query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index");
     query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2);
+    query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",10, "facet.limit",1, "facet.sort","index");
     query("q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1);
     query("q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*");
     query("q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1);