You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by ma...@apache.org on 2010/03/14 21:58:34 UTC

svn commit: r922957 [3/3] - in /lucene/solr/branches/solr: ./ lib/ src/common/org/apache/solr/common/util/ src/java/org/apache/solr/analysis/ src/java/org/apache/solr/handler/ src/java/org/apache/solr/handler/admin/ src/java/org/apache/solr/handler/com...

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/QueryResultKey.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/QueryResultKey.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/QueryResultKey.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/QueryResultKey.java Sun Mar 14 20:58:32 2010
@@ -17,10 +17,14 @@
 
 package org.apache.solr.search;
 
+import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.SolrException;
 
+import java.io.IOException;
 import java.util.List;
 
 /** A hash key encapsulating a query, a list of filters, and a sort
@@ -38,7 +42,7 @@ public final class QueryResultKey {
   private static SortField[] defaultSort = new SortField[0];
 
 
-  public QueryResultKey(Query query, List<Query> filters, Sort sort, int nc_flags) {
+  public QueryResultKey(Query query, List<Query> filters, Sort sort, int nc_flags) throws IOException {
     this.query = query;
     this.sort = sort;
     this.filters = filters;

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Sun Mar 14 20:58:32 2010
@@ -133,20 +133,10 @@ public class SolrConstantScoreQuery exte
       }
     }
 
-    /** @deprecated use {@link #nextDoc()} instead. */
-    public boolean next() throws IOException {
-      return docIdSetIterator.nextDoc() != NO_MORE_DOCS;
-    }
-
     public int nextDoc() throws IOException {
       return docIdSetIterator.nextDoc();
     }
 
-    /** @deprecated use {@link #docID()} instead. */
-    public int doc() {
-      return docIdSetIterator.doc();
-    }
-
     public int docID() {
       return docIdSetIterator.docID();
     }
@@ -155,11 +145,6 @@ public class SolrConstantScoreQuery exte
       return theScore;
     }
 
-    /** @deprecated use {@link #advance(int)} instead. */
-    public boolean skipTo(int target) throws IOException {
-      return docIdSetIterator.advance(target) != NO_MORE_DOCS;
-    }
-
     public int advance(int target) throws IOException {
       return docIdSetIterator.advance(target);
     }

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexReader.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexReader.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexReader.java Sun Mar 14 20:58:32 2010
@@ -455,11 +455,6 @@ public class SolrIndexReader extends Fil
   }
 
   @Override
-  public int getTermInfosIndexDivisor() {
-    return in.getTermInfosIndexDivisor();
-  }
-
-  @Override
   public void incRef() {
     in.incRef();
   }
@@ -480,11 +475,6 @@ public class SolrIndexReader extends Fil
   }
 
   @Override
-  public void setTermInfosIndexDivisor(int indexDivisor) throws IllegalStateException {
-    in.setTermInfosIndexDivisor(indexDivisor);
-  }
-
-  @Override
   public TermPositions termPositions(Term term) throws IOException {
     return in.termPositions(term);
   }
@@ -498,16 +488,6 @@ public class SolrIndexReader extends Fil
   public Object getFieldCacheKey() {
     return in.getFieldCacheKey();
   }
-
-  @Override
-  public boolean getDisableFakeNorms() {
-    return in.getDisableFakeNorms();
-  }
-
-  @Override
-  public void setDisableFakeNorms(boolean disableFakeNorms) {
-    in.setDisableFakeNorms(disableFakeNorms);
-  }
 }
 
 

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java Sun Mar 14 20:58:32 2010
@@ -342,22 +342,22 @@ public class SolrIndexSearcher extends I
     return qr;
   }
 
-  public Hits search(Query query, Filter filter, Sort sort) throws IOException {
-    // todo - when Solr starts accepting filters, need to
-    // change this conditional check (filter!=null) and create a new filter
-    // that ANDs them together if it already exists.
-
-    if (optimizer==null || filter!=null || !(query instanceof BooleanQuery)
-    ) {
-      return super.search(query,filter,sort);
-    } else {
-      Query[] newQuery = new Query[1];
-      Filter[] newFilter = new Filter[1];
-      optimizer.optimize((BooleanQuery)query, this, 0, newQuery, newFilter);
-
-      return super.search(newQuery[0], newFilter[0], sort);
-    }
-  }
+//  public Hits search(Query query, Filter filter, Sort sort) throws IOException {
+//    // todo - when Solr starts accepting filters, need to
+//    // change this conditional check (filter!=null) and create a new filter
+//    // that ANDs them together if it already exists.
+//
+//    if (optimizer==null || filter!=null || !(query instanceof BooleanQuery)
+//    ) {
+//      return super.search(query,filter,sort);
+//    } else {
+//      Query[] newQuery = new Query[1];
+//      Filter[] newFilter = new Filter[1];
+//      optimizer.optimize((BooleanQuery)query, this, 0, newQuery, newFilter);
+//
+//      return super.search(newQuery[0], newFilter[0], sort);
+//    }
+//  }
 
   /**
    * @return the indexDir on which this searcher is opened
@@ -697,10 +697,12 @@ public class SolrIndexSearcher extends I
   * This method is not cache-aware and no caches are checked.
   */
   public DocSet convertFilter(Filter lfilter) throws IOException {
-    BitSet bs = lfilter.bits(this.reader);
-    OpenBitSet obs = new OpenBitSet(bs.size());
-    for(int i=bs.nextSetBit(0); i>=0; i=bs.nextSetBit(i+1)) {
-      obs.fastSet(i);
+    DocIdSet docSet = lfilter.getDocIdSet(this.reader);
+    OpenBitSet obs = new OpenBitSet();
+    DocIdSetIterator it = docSet.iterator();
+    int doc;
+    while((doc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      obs.fastSet(doc);
     }
     return new BitDocSet(obs);
   }

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrQueryParser.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrQueryParser.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/SolrQueryParser.java Sun Mar 14 20:58:32 2010
@@ -25,15 +25,13 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.search.*;
+import org.apache.lucene.util.Version;
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.reverse.ReverseStringFilter;
 import org.apache.solr.analysis.*;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
-import org.apache.solr.schema.TrieField;
-import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.TextField;
 
 // TODO: implement the analysis of simple fields with
@@ -77,7 +75,7 @@ public class SolrQueryParser extends Que
    * @see IndexSchema#getDefaultSearchFieldName()
    */
   public SolrQueryParser(IndexSchema schema, String defaultField) {
-    super(defaultField == null ? schema.getDefaultSearchFieldName() : defaultField, schema.getQueryAnalyzer());
+    super(Version.LUCENE_24, defaultField == null ? schema.getDefaultSearchFieldName() : defaultField, schema.getQueryAnalyzer());
     this.schema = schema;
     this.parser  = null;
     this.defaultField = defaultField;
@@ -91,7 +89,7 @@ public class SolrQueryParser extends Que
   }
 
   public SolrQueryParser(QParser parser, String defaultField, Analyzer analyzer) {
-    super(defaultField, analyzer);
+    super(Version.LUCENE_24, defaultField, analyzer);
     this.schema = parser.getReq().getSchema();
     this.parser = parser;
     this.defaultField = defaultField;

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/WildcardFilter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/WildcardFilter.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/WildcardFilter.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/WildcardFilter.java Sun Mar 14 20:58:32 2010
@@ -26,7 +26,6 @@ import org.apache.lucene.index.TermEnum;
 import org.apache.lucene.index.TermDocs;
 import org.apache.lucene.util.OpenBitSet;
 
-import java.util.BitSet;
 import java.io.IOException;
 
 
@@ -43,20 +42,6 @@ public class WildcardFilter extends Filt
 
   public Term getTerm() { return term; }
 
-  /**
-   * @deprecated Use {@link #getDocIdSet(IndexReader)} instead.
-   */
-  @Override
-  public BitSet bits(IndexReader reader) throws IOException {
-    final BitSet bitSet = new BitSet(reader.maxDoc());
-    new WildcardGenerator(term) {
-      public void handleDoc(int doc) {
-        bitSet.set(doc);
-      }
-    }.generate(reader);
-    return bitSet;
-  }
-
   @Override
   public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
     final OpenBitSet bitSet = new OpenBitSet(reader.maxDoc());

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/search/function/FunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/search/function/FunctionQuery.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/search/function/FunctionQuery.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/search/function/FunctionQuery.java Sun Mar 14 20:58:32 2010
@@ -100,7 +100,7 @@ public class FunctionQuery extends Query
       int[] offsets = topReader.getLeafOffsets();
       int readerPos = SolrIndexReader.readerIndex(doc, offsets);
       int readerBase = offsets[readerPos];
-      return scorer(subReaders[readerPos], true, true).explain(doc-readerBase);
+      return ((AllScorer)scorer(subReaders[readerPos], true, true)).explain(doc-readerBase);
     }
   }
 

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/spelling/AbstractLuceneSpellChecker.java Sun Mar 14 20:58:32 2010
@@ -33,7 +33,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.search.spell.LevensteinDistance;
 import org.apache.lucene.search.spell.SpellChecker;
-import org.apache.lucene.search.spell.StringDistance;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.RAMDirectory;
@@ -184,7 +183,7 @@ public abstract class AbstractLuceneSpel
    */
   protected void initIndex() throws IOException {
     if (indexDir != null) {
-      index = FSDirectory.getDirectory(indexDir);
+      index = FSDirectory.open(new File(indexDir));
     } else {
       index = new RAMDirectory();
     }

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java Sun Mar 14 20:58:32 2010
@@ -22,12 +22,10 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.lucene.analysis.WhitespaceAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.spell.PlainTextDictionary;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.solr.common.util.NamedList;
@@ -98,7 +96,7 @@ public class FileBasedSpellChecker exten
 
         for (String s : lines) {
           Document d = new Document();
-          d.add(new Field(WORD_FIELD_NAME, s, Field.Store.NO, Field.Index.TOKENIZED));
+          d.add(new Field(WORD_FIELD_NAME, s, Field.Store.NO, Field.Index.ANALYZED));
           writer.addDocument(d);
         }
         writer.optimize();

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/spelling/IndexBasedSpellChecker.java Sun Mar 14 20:58:32 2010
@@ -16,17 +16,14 @@ package org.apache.solr.spelling;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.WhitespaceAnalyzer;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrResourceLoader;
-import org.apache.solr.schema.FieldType;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.HighFrequencyDictionary;
 
+import java.io.File;
 import java.io.IOException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -63,7 +60,7 @@ public class IndexBasedSpellChecker exte
   private void initSourceReader() {
     if (sourceLocation != null) {
       try {
-        FSDirectory luceneIndexDir = FSDirectory.getDirectory(sourceLocation);
+        FSDirectory luceneIndexDir = FSDirectory.open(new File(sourceLocation));
         this.reader = IndexReader.open(luceneIndexDir);
       } catch (IOException e) {
         throw new RuntimeException(e);

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/spelling/SpellingQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/spelling/SpellingQueryConverter.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/spelling/SpellingQueryConverter.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/spelling/SpellingQueryConverter.java Sun Mar 14 20:58:32 2010
@@ -27,6 +27,11 @@ import java.util.regex.Pattern;
 
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 
 
 /**
@@ -99,10 +104,22 @@ public class SpellingQueryConverter exte
       if (word.equals("AND") == false && word.equals("OR") == false) {
         try {
           stream = analyzer.reusableTokenStream("", new StringReader(word));
-          Token token;
-          while ((token = stream.next()) != null) {
+          // TODO: support custom attributes
+          TermAttribute termAtt = (TermAttribute) stream.addAttribute(TermAttribute.class);
+          FlagsAttribute flagsAtt = (FlagsAttribute) stream.addAttribute(FlagsAttribute.class);
+          TypeAttribute typeAtt = (TypeAttribute) stream.addAttribute(TypeAttribute.class);
+          PayloadAttribute payloadAtt = (PayloadAttribute) stream.addAttribute(PayloadAttribute.class);
+          PositionIncrementAttribute posIncAtt = (PositionIncrementAttribute) stream.addAttribute(PositionIncrementAttribute.class);
+          stream.reset();
+          while (stream.incrementToken()) {
+            Token token = new Token();
+            token.setTermBuffer(termAtt.termBuffer(), 0, termAtt.termLength());
             token.setStartOffset(matcher.start());
             token.setEndOffset(matcher.end());
+            token.setFlags(flagsAtt.getFlags());
+            token.setType(typeAtt.type());
+            token.setPayload(payloadAtt.getPayload());
+            token.setPositionIncrement(posIncAtt.getPositionIncrement());
             result.add(token);
           }
         } catch (IOException e) {

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/tst/OldRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/tst/OldRequestHandler.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/tst/OldRequestHandler.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/tst/OldRequestHandler.java Sun Mar 14 20:58:32 2010
@@ -66,12 +66,11 @@ public class OldRequestHandler implement
       sort = QueryParsing.parseSort(commands.get(1), req.getSchema());
     }
 
-    Hits hits=null;
 
     try {
-      hits = req.getSearcher().search(query,filter,sort);
+      TopFieldDocs hits = req.getSearcher().search(query,filter, req.getStart()+req.getLimit(), sort);
 
-      int numHits = hits.length();
+      int numHits = hits.totalHits;
       int startRow = Math.min(numHits, req.getStart());
       int endRow = Math.min(numHits,req.getStart()+req.getLimit());
       int numRows = endRow-startRow;
@@ -79,8 +78,8 @@ public class OldRequestHandler implement
       int[] ids = new int[numRows];
       Document[] data = new Document[numRows];
       for (int i=startRow; i<endRow; i++) {
-        ids[i] = hits.id(i);
-        data[i] = hits.doc(i);
+        ids[i] = hits.scoreDocs[i].doc;
+        data[i] = req.getSearcher().doc(ids[i]);
       }
 
       rsp.add(null, new DocSlice(0,numRows,ids,null,numHits,0.0f));

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/tst/TestRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/tst/TestRequestHandler.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/tst/TestRequestHandler.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/tst/TestRequestHandler.java Sun Mar 14 20:58:32 2010
@@ -144,7 +144,7 @@ public class TestRequestHandler implemen
         nl.add("myLong",999999999999L);
 
         Document doc = new Document();
-        doc.add(new Field("id","55",Field.Store.YES, Field.Index.UN_TOKENIZED));
+        doc.add(new Field("id","55",Field.Store.YES, Field.Index.NOT_ANALYZED));
         nl.add("myDoc",doc);
 
         nl.add("myResult",results);
@@ -172,8 +172,8 @@ public class TestRequestHandler implemen
       //
       // test against hits
       //
-      Hits hits = searcher.search(query, lfilter, sort);
-      test(hits.length() == results.matches());
+      TopFieldDocs hits = searcher.search(query, lfilter, 1000, sort);
+      test(hits.totalHits == results.matches());
 
 
       DocList rrr2 = results.subset(start,limit);
@@ -189,7 +189,7 @@ public class TestRequestHandler implemen
       ***/
 
       for (int i=0; i<results.size(); i++) {
-        test( iter.nextDoc() == hits.id(i+results.offset()) );
+        test( iter.nextDoc() == hits.scoreDocs[i].doc);
 
         // Document doesn't implement equals()
         // test( searcher.document(i).equals(hits.doc(i)));

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/update/SolrIndexWriter.java Sun Mar 14 20:58:32 2010
@@ -161,29 +161,32 @@ public class SolrIndexWriter extends Ind
    *
    */
   public SolrIndexWriter(String name, String path, DirectoryFactory dirFactory, boolean create, IndexSchema schema) throws IOException {
-    super(getDirectory(path, dirFactory, null), false, schema.getAnalyzer(), create);
+    super(getDirectory(path, dirFactory, null), schema.getAnalyzer(), create, MaxFieldLength.LIMITED);
     init(name, schema, null);
   }
 
   @Deprecated
+  // nocommit: remove?
   public SolrIndexWriter(String name, String path, DirectoryFactory dirFactory, boolean create, IndexSchema schema, SolrIndexConfig config) throws IOException {
-    super(getDirectory(path, dirFactory, null), config.luceneAutoCommit, schema.getAnalyzer(), create);
+    super(getDirectory(path, dirFactory, null), schema.getAnalyzer(), create, MaxFieldLength.LIMITED);
     init(name, schema, config);
   }
   
   /**
    * @deprecated
    */
+  // nocommit: remove?
   public SolrIndexWriter(String name, String path, boolean create, IndexSchema schema) throws IOException {
-    super(getDirectory(path, null), false, schema.getAnalyzer(), create);
+    super(getDirectory(path, null), schema.getAnalyzer(), create, MaxFieldLength.LIMITED);
     init(name, schema, null);
   }
 
   /**
    * @deprecated
    */
+  // nocommit: remove?
   public SolrIndexWriter(String name, String path, boolean create, IndexSchema schema, SolrIndexConfig config) throws IOException {
-    super(getDirectory(path, config), config.luceneAutoCommit, schema.getAnalyzer(), create);
+    super(getDirectory(path, config), schema.getAnalyzer(), create, MaxFieldLength.LIMITED);
     init(name, schema, config);
   }
 

Modified: lucene/solr/branches/solr/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/java/org/apache/solr/update/UpdateHandler.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/solr/branches/solr/src/java/org/apache/solr/update/UpdateHandler.java Sun Mar 14 20:58:32 2010
@@ -18,11 +18,13 @@
 package org.apache.solr.update;
 
 
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
-import org.apache.lucene.search.HitCollector;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.Scorer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -135,17 +137,19 @@ public abstract class UpdateHandler impl
   public abstract void close() throws IOException;
 
 
-  static class DeleteHitCollector extends HitCollector {
+  static class DeleteHitCollector extends Collector {
     public int deleted=0;
     public final SolrIndexSearcher searcher;
+    private int docBase;
 
     public DeleteHitCollector(SolrIndexSearcher searcher) {
       this.searcher = searcher;
     }
 
-    public void collect(int doc, float score) {
+    @Override
+    public void collect(int doc) {
       try {
-        searcher.getReader().deleteDocument(doc);
+        searcher.getReader().deleteDocument(doc + docBase);
         deleted++;
       } catch (IOException e) {
         // don't try to close the searcher on failure for now...
@@ -153,6 +157,21 @@ public abstract class UpdateHandler impl
         throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error deleting doc# "+doc,e,false);
       }
     }
+
+    @Override
+    public boolean acceptsDocsOutOfOrder() {
+      return false;
+    }
+
+    @Override
+    public void setNextReader(IndexReader arg0, int docBase) throws IOException {
+      this.docBase = docBase;
+    }
+
+    @Override
+    public void setScorer(Scorer scorer) throws IOException {
+      
+    }
   }
 
 

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/BasicFunctionalityTest.java Sun Mar 14 20:58:32 2010
@@ -538,21 +538,21 @@ public class BasicFunctionalityTest exte
     SchemaField f; // Solr field type
     Field luf; // Lucene field
 
-    f = ischema.getField("test_hlt");
-    luf = f.createField("test", 0f);
-    assertFalse(luf.isCompressed());
-    assertTrue(luf.isStored());
-
-    f = ischema.getField("test_hlt");
-    luf = f.createField(mkstr(345), 0f);
-    assertTrue(luf.isCompressed());
-    assertTrue(luf.isStored());
-
-    f = ischema.getField("test_hlt_off");
-    luf = f.createField(mkstr(400), 0f);
-    assertFalse(luf.isCompressed());
-    assertTrue(luf.isStored());
-    
+//    f = ischema.getField("test_hlt");
+//    luf = f.createField("test", 0f);
+//    assertFalse(luf.isCompressed());
+//    assertTrue(luf.isStored());
+//
+//    f = ischema.getField("test_hlt");
+//    luf = f.createField(mkstr(345), 0f);
+//    assertTrue(luf.isCompressed());
+//    assertTrue(luf.isStored());
+//
+//    f = ischema.getField("test_hlt_off");
+//    luf = f.createField(mkstr(400), 0f);
+//    assertFalse(luf.isCompressed());
+//    assertTrue(luf.isStored());
+//    
   }
 
   public void testNotLazyField() throws IOException {

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/analysis/TestBufferedTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/analysis/TestBufferedTokenStream.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/analysis/TestBufferedTokenStream.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/analysis/TestBufferedTokenStream.java Sun Mar 14 20:58:32 2010
@@ -37,7 +37,7 @@ public class TestBufferedTokenStream ext
     protected Token process(Token t) throws IOException {
       if ("A".equals(new String(t.termBuffer(), 0, t.termLength()))) {
         Token t2 = read();
-        if (t2!=null && "B".equals(new String(t2.termBuffer(), 0, t2.termLength()))) t.setTermText("Q");
+        if (t2!=null && "B".equals(new String(t2.termBuffer(), 0, t2.termLength()))) t.setTermBuffer("Q");
         if (t2!=null) pushBack(t2);
       }
       return t;

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/core/AlternateDirectoryTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/core/AlternateDirectoryTest.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/core/AlternateDirectoryTest.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/core/AlternateDirectoryTest.java Sun Mar 14 20:58:32 2010
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.core;
 
+import java.io.File;
 import java.io.IOException;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.solr.util.AbstractSolrTestCase;
@@ -44,7 +45,7 @@ public class AlternateDirectoryTest exte
 
     public FSDirectory open(String path) throws IOException {
       openCalled = true;
-      return FSDirectory.getDirectory(path);
+      return FSDirectory.open(new File(path));
     }
 
   }

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/core/TestArbitraryIndexDir.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/core/TestArbitraryIndexDir.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/core/TestArbitraryIndexDir.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/core/TestArbitraryIndexDir.java Sun Mar 14 20:58:32 2010
@@ -30,11 +30,11 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.IndexWriter.MaxFieldLength;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.queryParser.QueryParser;
-import org.apache.lucene.search.Hits;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.util.TestHarness;
@@ -97,11 +97,11 @@ public class TestArbitraryIndexDir exten
     }
 
     //add a doc in the new index dir
-    Directory dir = FSDirectory.getDirectory(newDir);
-    IndexWriter iw = new IndexWriter(dir, new StandardAnalyzer(), new MaxFieldLength(1000));
+    Directory dir = FSDirectory.open(newDir);
+    IndexWriter iw = new IndexWriter(dir, new StandardAnalyzer(Version.LUCENE_24), new MaxFieldLength(1000));
     Document doc = new Document();
-    doc.add(new Field("id", "2", Field.Store.YES, Field.Index.TOKENIZED));
-    doc.add(new Field("name", "name2", Field.Store.YES, Field.Index.TOKENIZED));
+    doc.add(new Field("id", "2", Field.Store.YES, Field.Index.ANALYZED));
+    doc.add(new Field("name", "name2", Field.Store.YES, Field.Index.ANALYZED));
     iw.addDocument(doc);
     iw.commit();
     iw.close();

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/highlight/HighlighterTest.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/highlight/HighlighterTest.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/highlight/HighlighterTest.java Sun Mar 14 20:58:32 2010
@@ -158,12 +158,11 @@ public class HighlighterTest extends Abs
       TokenStream ts1 = tots.getMultiValuedTokenStream( v.length() );
       Analyzer a2 = new WhitespaceAnalyzer();
       TokenStream ts2 = a2.tokenStream( "", new StringReader( v ) );
-      Token t1 = new Token();
-      Token t2 = new Token();
-      for( t1 = ts1.next( t1 ); t1 != null; t1 = ts1.next( t1 ) ){
-        t2 = ts2.next( t2 );
-        assertEquals( t2, t1 );
+      while (ts1.incrementToken()) {
+        assertTrue(ts2.incrementToken());
+        assertEquals(ts1, ts2);
       }
+      assertFalse(ts2.incrementToken());
     }
   }
 

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/spelling/IndexBasedSpellCheckerTest.java Sun Mar 14 20:58:32 2010
@@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.spell.JaroWinklerDistance;
 import org.apache.lucene.search.spell.SpellChecker;
 import org.apache.lucene.search.spell.StringDistance;
+import org.apache.lucene.store.FSDirectory;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.util.AbstractSolrTestCase;
@@ -254,10 +255,10 @@ public class IndexBasedSpellCheckerTest 
     File indexDir = new File(tmpDir, "spellingIdx" + new Date().getTime());
     //create a standalone index
     File altIndexDir = new File(tmpDir, "alternateIdx" + new Date().getTime());
-    IndexWriter iw = new IndexWriter(altIndexDir, new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
+    IndexWriter iw = new IndexWriter(FSDirectory.open(altIndexDir), new WhitespaceAnalyzer(), IndexWriter.MaxFieldLength.LIMITED);
     for (int i = 0; i < ALT_DOCS.length; i++) {
       Document doc = new Document();
-      doc.add(new Field("title", ALT_DOCS[i], Field.Store.YES, Field.Index.TOKENIZED));
+      doc.add(new Field("title", ALT_DOCS[i], Field.Store.YES, Field.Index.ANALYZED));
       iw.addDocument(doc);
     }
     iw.optimize();

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/spelling/SimpleQueryConverter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/spelling/SimpleQueryConverter.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/spelling/SimpleQueryConverter.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/spelling/SimpleQueryConverter.java Sun Mar 14 20:58:32 2010
@@ -19,6 +19,12 @@ package org.apache.solr.spelling;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.WhitespaceAnalyzer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
+import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 
 import java.util.Collection;
 import java.util.HashSet;
@@ -36,9 +42,24 @@ class SimpleQueryConverter extends Spell
     Collection<Token> result = new HashSet<Token>();
     WhitespaceAnalyzer analyzer = new WhitespaceAnalyzer();
     TokenStream ts = analyzer.tokenStream("", new StringReader(origQuery));
-    Token tok = null;
+    // TODO: support custom attributes
+    TermAttribute termAtt = (TermAttribute) ts.addAttribute(TermAttribute.class);
+    OffsetAttribute offsetAtt = (OffsetAttribute) ts.addAttribute(OffsetAttribute.class);
+    TypeAttribute typeAtt = (TypeAttribute) ts.addAttribute(TypeAttribute.class);
+    FlagsAttribute flagsAtt = (FlagsAttribute) ts.addAttribute(FlagsAttribute.class);
+    PayloadAttribute payloadAtt = (PayloadAttribute) ts.addAttribute(PayloadAttribute.class);
+    PositionIncrementAttribute posIncAtt = (PositionIncrementAttribute) ts.addAttribute(PositionIncrementAttribute.class);
+    
     try {
-      while ((tok = ts.next()) != null){
+      ts.reset();
+      while (ts.incrementToken()){
+        Token tok = new Token();
+        tok.setTermBuffer(termAtt.termBuffer(), 0, termAtt.termLength());
+        tok.setOffset(offsetAtt.startOffset(), offsetAtt.endOffset());
+        tok.setFlags(flagsAtt.getFlags());
+        tok.setPayload(payloadAtt.getPayload());
+        tok.setPositionIncrement(posIncAtt.getPositionIncrement());
+        tok.setType(typeAtt.type());
         result.add(tok);
       }
     } catch (IOException e) {

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerOptimizeTest.java Sun Mar 14 20:58:32 2010
@@ -53,8 +53,8 @@ public class DirectUpdateHandlerOptimize
     for (int i = 0; i < 99; i++) {
       // Add a valid document
       cmd.doc = new Document();
-      cmd.doc.add(new Field("id", "id_" + i, Field.Store.YES, Field.Index.UN_TOKENIZED));
-      cmd.doc.add(new Field("subject", "subject_" + i, Field.Store.NO, Field.Index.TOKENIZED));
+      cmd.doc.add(new Field("id", "id_" + i, Field.Store.YES, Field.Index.NOT_ANALYZED));
+      cmd.doc.add(new Field("subject", "subject_" + i, Field.Store.NO, Field.Index.ANALYZED));
       updater.addDoc(cmd);
     }
 

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java Sun Mar 14 20:58:32 2010
@@ -67,16 +67,16 @@ public class DirectUpdateHandlerTest ext
     
     // Add a valid document
     cmd.doc = new Document();
-    cmd.doc.add( new Field( "id", "AAA", Store.YES, Index.UN_TOKENIZED ) );
-    cmd.doc.add( new Field( "subject", "xxxxx", Store.YES, Index.UN_TOKENIZED ) );
+    cmd.doc.add( new Field( "id", "AAA", Store.YES, Index.NOT_ANALYZED ) );
+    cmd.doc.add( new Field( "subject", "xxxxx", Store.YES, Index.NOT_ANALYZED ) );
     updater.addDoc( cmd );
     
     // Add a document with multiple ids
     cmd.indexedId = null;  // reset the id for this add
     cmd.doc = new Document();
-    cmd.doc.add( new Field( "id", "AAA", Store.YES, Index.UN_TOKENIZED ) );
-    cmd.doc.add( new Field( "id", "BBB", Store.YES, Index.UN_TOKENIZED ) );
-    cmd.doc.add( new Field( "subject", "xxxxx", Store.YES, Index.UN_TOKENIZED ) );
+    cmd.doc.add( new Field( "id", "AAA", Store.YES, Index.NOT_ANALYZED ) );
+    cmd.doc.add( new Field( "id", "BBB", Store.YES, Index.NOT_ANALYZED ) );
+    cmd.doc.add( new Field( "subject", "xxxxx", Store.YES, Index.NOT_ANALYZED ) );
     try {
       updater.addDoc( cmd );
       fail( "added a document with multiple ids" );
@@ -86,7 +86,7 @@ public class DirectUpdateHandlerTest ext
     // Add a document without an id
     cmd.indexedId = null;  // reset the id for this add
     cmd.doc = new Document();
-    cmd.doc.add( new Field( "subject", "xxxxx", Store.YES, Index.UN_TOKENIZED ) );
+    cmd.doc.add( new Field( "subject", "xxxxx", Store.YES, Index.NOT_ANALYZED ) );
     try {
       updater.addDoc( cmd );
       fail( "added a document without an ids" );
@@ -325,7 +325,7 @@ public class DirectUpdateHandlerTest ext
     
     // Add a document
     cmd.doc = new Document();
-    cmd.doc.add( new Field( "id", id, Store.YES, Index.UN_TOKENIZED ) );
+    cmd.doc.add( new Field( "id", id, Store.YES, Index.NOT_ANALYZED ) );
     updater.addDoc( cmd );
   }
   

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/util/TestCharArrayMap.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/util/TestCharArrayMap.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/util/TestCharArrayMap.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/util/TestCharArrayMap.java Sun Mar 14 20:58:32 2010
@@ -126,11 +126,12 @@ public class TestCharArrayMap extends Te
 
     int ret=0;
     long start = System.currentTimeMillis();
-    String[] stopwords = StopAnalyzer.ENGLISH_STOP_WORDS;
+    Set<String> stopwords = (Set<String>) StopAnalyzer.ENGLISH_STOP_WORDS_SET;
     // words = "this is a different test to see what is really going on here... I hope it works well but I'm not sure it will".split(" ");
-    char[][] stopwordschars = new char[stopwords.length][];
-    for (int i=0; i<stopwords.length; i++) {
-      stopwordschars[i] = stopwords[i].toCharArray();
+    char[][] stopwordschars = new char[stopwords.size()][];
+    Iterator<String> it = stopwords.iterator();
+    for (int i=0; i<stopwords.size(); i++) {
+      stopwordschars[i] = it.next().toCharArray();
     }
 
     String[] testwords = "now is the time for all good men to come to the aid of their country".split(" ");

Modified: lucene/solr/branches/solr/src/test/org/apache/solr/util/TestOpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/solr/src/test/org/apache/solr/util/TestOpenBitSet.java?rev=922957&r1=922956&r2=922957&view=diff
==============================================================================
--- lucene/solr/branches/solr/src/test/org/apache/solr/util/TestOpenBitSet.java (original)
+++ lucene/solr/branches/solr/src/test/org/apache/solr/util/TestOpenBitSet.java Sun Mar 14 20:58:32 2010
@@ -57,11 +57,11 @@ public class TestOpenBitSet extends Test
     do {
       aa = a.nextSetBit(aa+1);
       if (rand.nextBoolean()) {
-        iterator.next();
-        bb = iterator.doc();
+        iterator.nextDoc();
+        bb = iterator.docID();
       } else {
-        iterator.skipTo(bb+1);
-        bb = iterator.doc();
+        iterator.advance(bb+1);
+        bb = iterator.docID();
       }
       assertEquals(aa == -1 ? DocIdSetIterator.NO_MORE_DOCS : aa, bb);
     } while (aa>=0);