You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/07/05 23:37:21 UTC

svn commit: r1357938 [2/2] - in /lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene: codecs/ codecs/lucene40/ codecs/lucene40/values/ codecs/simpletext/ document/ index/ search/ util/

Added: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableField.java?rev=1357938&view=auto
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableField.java (added)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableField.java Thu Jul  5 21:37:19 2012
@@ -0,0 +1,44 @@
+package org.apache.lucene.index;
+
+import java.io.Reader;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public interface StorableField {
+
+  /** Field name */
+  public String name();
+  
+  /** Field type */
+  public FieldType fieldType();
+  
+  /** Non-null if this field has a binary value */
+  public BytesRef binaryValue();
+
+  /** Non-null if this field has a string value */
+  public String stringValue();
+
+  /** Non-null if this field has a Reader value */
+  public Reader readerValue();
+
+  /** Non-null if this field has a numeric value */
+  public Number numericValue();
+}

Added: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableFieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableFieldType.java?rev=1357938&view=auto
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableFieldType.java (added)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StorableFieldType.java Thu Jul  5 21:37:19 2012
@@ -0,0 +1,25 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public interface StorableFieldType {
+
+  /** DocValues type; if non-null then the field's value
+   *  will be indexed into docValues */
+  public DocValues.Type docValueType();
+}

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java?rev=1357938&r1=1357937&r2=1357938&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java Thu Jul  5 21:37:19 2012
@@ -44,12 +44,12 @@ final class StoredFieldsConsumer {
   }
 
   private int numStoredFields;
-  private IndexableField[] storedFields;
+  private StorableField[] storedFields;
   private FieldInfo[] fieldInfos;
 
   public void reset() {
     numStoredFields = 0;
-    storedFields = new IndexableField[1];
+    storedFields = new StorableField[1];
     fieldInfos = new FieldInfo[1];
   }
 
@@ -126,10 +126,10 @@ final class StoredFieldsConsumer {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
-  public void addField(IndexableField field, FieldInfo fieldInfo) {
+  public void addField(StorableField field, FieldInfo fieldInfo) {
     if (numStoredFields == storedFields.length) {
       int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
-      IndexableField[] newArray = new IndexableField[newSize];
+      StorableField[] newArray = new StorableField[newSize];
       System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
       storedFields = newArray;
       

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java?rev=1357938&r1=1357937&r2=1357938&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java Thu Jul  5 21:37:19 2012
@@ -61,7 +61,7 @@ final class TermVectorsConsumerPerField 
 
     for(int i=0;i<count;i++) {
       IndexableField field = fields[i];
-      if (field.fieldType().indexed() && field.fieldType().storeTermVectors()) {
+      if (field.fieldType().storeTermVectors()) {
         doVectors = true;
         doVectorPositions |= field.fieldType().storeTermVectorPositions();
         doVectorOffsets |= field.fieldType().storeTermVectorOffsets();

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1357938&r1=1357937&r2=1357938&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Thu Jul  5 21:37:19 2012
@@ -33,6 +33,7 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.StoredDocument;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader; // javadocs
 import org.apache.lucene.index.IndexReader;
@@ -181,7 +182,7 @@ public class IndexSearcher {
   }
 
   /** Sugar for <code>.getIndexReader().document(docID)</code> */
-  public Document doc(int docID) throws IOException {
+  public StoredDocument doc(int docID) throws IOException {
     return reader.document(docID);
   }
 
@@ -191,7 +192,7 @@ public class IndexSearcher {
   }
 
   /** Sugar for <code>.getIndexReader().document(docID, fieldsToLoad)</code> */
-  public final Document document(int docID, Set<String> fieldsToLoad) throws IOException {
+  public final StoredDocument document(int docID, Set<String> fieldsToLoad) throws IOException {
     return reader.document(docID, fieldsToLoad);
   }
 

Modified: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/NRTManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/NRTManager.java?rev=1357938&r1=1357937&r2=1357938&view=diff
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/NRTManager.java (original)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/search/NRTManager.java Thu Jul  5 21:37:19 2012
@@ -27,6 +27,7 @@ import java.util.concurrent.locks.Reentr
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexDocument;
 import org.apache.lucene.index.IndexReader; // javadocs
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexableField;
@@ -153,25 +154,25 @@ public class NRTManager extends Referenc
       this.writer = writer;
     }
 
-    public long updateDocument(Term t, Iterable<? extends IndexableField> d, Analyzer a) throws IOException {
+    public long updateDocument(Term t, IndexDocument d, Analyzer a) throws IOException {
       writer.updateDocument(t, d, a);
       // Return gen as of when indexing finished:
       return indexingGen.get();
     }
 
-    public long updateDocument(Term t, Iterable<? extends IndexableField> d) throws IOException {
+    public long updateDocument(Term t, IndexDocument d) throws IOException {
       writer.updateDocument(t, d);
       // Return gen as of when indexing finished:
       return indexingGen.get();
     }
 
-    public long updateDocuments(Term t, Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer a) throws IOException {
+    public long updateDocuments(Term t, Iterable<? extends IndexDocument> docs, Analyzer a) throws IOException {
       writer.updateDocuments(t, docs, a);
       // Return gen as of when indexing finished:
       return indexingGen.get();
     }
 
-    public long updateDocuments(Term t, Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
+    public long updateDocuments(Term t, Iterable<? extends IndexDocument> docs) throws IOException {
       writer.updateDocuments(t, docs);
       // Return gen as of when indexing finished:
       return indexingGen.get();
@@ -207,25 +208,25 @@ public class NRTManager extends Referenc
       return indexingGen.get();
     }
 
-    public long addDocument(Iterable<? extends IndexableField> d, Analyzer a) throws IOException {
+    public long addDocument(IndexDocument d, Analyzer a) throws IOException {
       writer.addDocument(d, a);
       // Return gen as of when indexing finished:
       return indexingGen.get();
     }
 
-    public long addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer a) throws IOException {
+    public long addDocuments(Iterable<? extends IndexDocument> docs, Analyzer a) throws IOException {
       writer.addDocuments(docs, a);
       // Return gen as of when indexing finished:
       return indexingGen.get();
     }
 
-    public long addDocument(Iterable<? extends IndexableField> d) throws IOException {
+    public long addDocument(IndexDocument d) throws IOException {
       writer.addDocument(d);
       // Return gen as of when indexing finished:
       return indexingGen.get();
     }
 
-    public long addDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs) throws IOException {
+    public long addDocuments(Iterable<? extends IndexDocument> docs) throws IOException {
       writer.addDocuments(docs);
       // Return gen as of when indexing finished:
       return indexingGen.get();

Added: lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java?rev=1357938&view=auto
==============================================================================
--- lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java (added)
+++ lucene/dev/branches/lucene3312/lucene/core/src/java/org/apache/lucene/util/FilterIterator.java Thu Jul  5 21:37:19 2012
@@ -0,0 +1,68 @@
+package org.apache.lucene.util;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+public abstract class FilterIterator<T, U extends T> implements Iterator<T> {
+  
+  private Iterator<U> iterator;
+  private T next = null;
+  private boolean nextIsSet = false;
+  
+  protected abstract boolean predicateFunction(U field);
+  
+  public FilterIterator(Iterator<U> baseIterator) {
+    this.iterator = baseIterator;
+  }
+  
+  public boolean hasNext() {
+    if (nextIsSet) {
+      return true;
+    } else {
+      return setNext();
+    }
+  }
+  
+  public T next() {
+    if (!nextIsSet) {
+      if (!setNext()) {
+        throw new NoSuchElementException();
+      }
+    }
+    nextIsSet = false;
+    return next;
+  }
+  
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+  
+  private boolean setNext() {
+    while (iterator.hasNext()) {
+      U object = iterator.next();
+      if (predicateFunction(object)) {
+        next = object;
+        nextIsSet = true;
+        return true;
+      }
+    }
+    return false;
+  }
+}