LUCENE-1257: More generified APIs and implementations in index package. Thanks Kay Kay!

git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@826389 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uwe Schindler 2009-10-18 09:50:49 +00:00
parent 9ac3cf9203
commit 290762a68e
30 changed files with 209 additions and 246 deletions

View File

@ -22,7 +22,7 @@ import java.util.Collection;
abstract class DocConsumer { abstract class DocConsumer {
abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException; abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException;
abstract void flush(final Collection<DocFieldProcessorPerThread> threads, final SegmentWriteState state) throws IOException; abstract void flush(final Collection<DocConsumerPerThread> threads, final SegmentWriteState state) throws IOException;
abstract void closeDocStore(final SegmentWriteState state) throws IOException; abstract void closeDocStore(final SegmentWriteState state) throws IOException;
abstract void abort(); abstract void abort();
abstract boolean freeRAM(); abstract boolean freeRAM();

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.Map; import java.util.Map;
abstract class DocFieldConsumer { abstract class DocFieldConsumer {
@ -26,7 +27,7 @@ abstract class DocFieldConsumer {
/** Called when DocumentsWriter decides to create a new /** Called when DocumentsWriter decides to create a new
* segment */ * segment */
abstract void flush(Map threadsAndFields, SegmentWriteState state) throws IOException; abstract void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
/** Called when DocumentsWriter decides to close the doc /** Called when DocumentsWriter decides to close the doc
* stores */ * stores */

View File

@ -44,7 +44,7 @@ final class DocFieldConsumers extends DocFieldConsumer {
two.setFieldInfos(fieldInfos); two.setFieldInfos(fieldInfos);
} }
public void flush(Map threadsAndFields, SegmentWriteState state) throws IOException { public void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
Map oneThreadsAndFields = new HashMap(); Map oneThreadsAndFields = new HashMap();
Map twoThreadsAndFields = new HashMap(); Map twoThreadsAndFields = new HashMap();

View File

@ -50,10 +50,11 @@ final class DocFieldProcessor extends DocConsumer {
fieldsWriter.closeDocStore(state); fieldsWriter.closeDocStore(state);
} }
public void flush(Collection<DocFieldProcessorPerThread> threads, SegmentWriteState state) throws IOException { public void flush(Collection<DocConsumerPerThread> threads, SegmentWriteState state) throws IOException {
Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> childThreadsAndFields = new HashMap<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>>(); Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> childThreadsAndFields = new HashMap<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>>();
for ( DocFieldProcessorPerThread perThread : threads) { for ( DocConsumerPerThread thread : threads) {
DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread;
childThreadsAndFields.put(perThread.consumer, perThread.fields()); childThreadsAndFields.put(perThread.consumer, perThread.fields());
perThread.trimFields(state); perThread.trimFields(state);
} }

View File

@ -21,10 +21,9 @@ import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.Map; import java.util.Map;
import org.apache.lucene.util.AttributeSource;
/** This is a DocFieldConsumer that inverts each field, /** This is a DocFieldConsumer that inverts each field,
* separately, from a Document, and accepts a * separately, from a Document, and accepts a
@ -46,25 +45,20 @@ final class DocInverter extends DocFieldConsumer {
endConsumer.setFieldInfos(fieldInfos); endConsumer.setFieldInfos(fieldInfos);
} }
void flush(Map threadsAndFields, SegmentWriteState state) throws IOException { void flush(Map<DocFieldConsumerPerThread, Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
Map childThreadsAndFields = new HashMap(); Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> childThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
Map endChildThreadsAndFields = new HashMap(); Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> endChildThreadsAndFields = new HashMap<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>>();
Iterator it = threadsAndFields.entrySet().iterator(); for (Map.Entry<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> entry : threadsAndFields.entrySet() ) {
while(it.hasNext()) {
Map.Entry entry = (Map.Entry) it.next();
DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey(); DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey();
Collection fields = (Collection) entry.getValue(); Collection<InvertedDocConsumerPerField> childFields = new HashSet<InvertedDocConsumerPerField>();
Collection<InvertedDocEndConsumerPerField> endChildFields = new HashSet<InvertedDocEndConsumerPerField>();
Iterator fieldsIt = fields.iterator(); for (final DocFieldConsumerPerField field: entry.getValue() ) {
Collection childFields = new HashSet(); DocInverterPerField perField = (DocInverterPerField) field;
Collection endChildFields = new HashSet();
while(fieldsIt.hasNext()) {
DocInverterPerField perField = (DocInverterPerField) fieldsIt.next();
childFields.add(perField.consumer); childFields.add(perField.consumer);
endChildFields.add(perField.endConsumer); endChildFields.add(perField.endConsumer);
} }

View File

@ -24,7 +24,6 @@ import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map.Entry; import java.util.Map.Entry;
@ -124,7 +123,7 @@ final class DocumentsWriter {
// than this they share ThreadStates // than this they share ThreadStates
private final static int MAX_THREAD_STATE = 5; private final static int MAX_THREAD_STATE = 5;
private DocumentsWriterThreadState[] threadStates = new DocumentsWriterThreadState[0]; private DocumentsWriterThreadState[] threadStates = new DocumentsWriterThreadState[0];
private final HashMap threadBindings = new HashMap(); private final HashMap<Thread,DocumentsWriterThreadState> threadBindings = new HashMap<Thread,DocumentsWriterThreadState>();
private int pauseThreads; // Non-zero when we need all threads to private int pauseThreads; // Non-zero when we need all threads to
// pause (eg to flush) // pause (eg to flush)
@ -138,7 +137,7 @@ final class DocumentsWriter {
int maxFieldLength = IndexWriter.DEFAULT_MAX_FIELD_LENGTH; int maxFieldLength = IndexWriter.DEFAULT_MAX_FIELD_LENGTH;
Similarity similarity; Similarity similarity;
List newFiles; List<String> newFiles;
static class DocState { static class DocState {
DocumentsWriter docWriter; DocumentsWriter docWriter;
@ -383,11 +382,11 @@ final class DocumentsWriter {
} }
} }
private Collection abortedFiles; // List of files that were written before last abort() private Collection<String> abortedFiles; // List of files that were written before last abort()
private SegmentWriteState flushState; private SegmentWriteState flushState;
Collection abortedFiles() { Collection<String> abortedFiles() {
return abortedFiles; return abortedFiles;
} }
@ -396,17 +395,17 @@ final class DocumentsWriter {
writer.message("DW: " + message); writer.message("DW: " + message);
} }
final List openFiles = new ArrayList(); final List<String> openFiles = new ArrayList<String>();
final List closedFiles = new ArrayList(); final List<String> closedFiles = new ArrayList<String>();
/* Returns Collection of files in use by this instance, /* Returns Collection of files in use by this instance,
* including any flushed segments. */ * including any flushed segments. */
synchronized List openFiles() { synchronized List<String> openFiles() {
return (List) ((ArrayList) openFiles).clone(); return ( List<String>) ((ArrayList<String>) openFiles).clone();
} }
synchronized List closedFiles() { synchronized List<String> closedFiles() {
return (List) ((ArrayList) closedFiles).clone(); return (List<String>) ((ArrayList<String>) closedFiles).clone();
} }
synchronized void addOpenFile(String name) { synchronized void addOpenFile(String name) {
@ -576,7 +575,7 @@ final class DocumentsWriter {
flushState.numDocsInStore = 0; flushState.numDocsInStore = 0;
} }
Collection threads = new HashSet(); Collection<DocConsumerPerThread> threads = new HashSet<DocConsumerPerThread>();
for(int i=0;i<threadStates.length;i++) for(int i=0;i<threadStates.length;i++)
threads.add(threadStates[i].consumer); threads.add(threadStates[i].consumer);
consumer.flush(threads, flushState); consumer.flush(threads, flushState);
@ -611,9 +610,8 @@ final class DocumentsWriter {
void createCompoundFile(String segment) throws IOException { void createCompoundFile(String segment) throws IOException {
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segment + "." + IndexFileNames.COMPOUND_FILE_EXTENSION); CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segment + "." + IndexFileNames.COMPOUND_FILE_EXTENSION);
Iterator it = flushState.flushedFiles.iterator(); for (final String flushedFile : flushState.flushedFiles)
while(it.hasNext()) cfsWriter.addFile(flushedFile);
cfsWriter.addFile((String) it.next());
// Perform the merge // Perform the merge
cfsWriter.close(); cfsWriter.close();
@ -828,7 +826,7 @@ final class DocumentsWriter {
} }
// for testing // for testing
synchronized HashMap getBufferedDeleteTerms() { synchronized HashMap<Term,BufferedDeletes.Num> getBufferedDeleteTerms() {
return deletesInRAM.terms; return deletesInRAM.terms;
} }
@ -1191,7 +1189,7 @@ final class DocumentsWriter {
private class ByteBlockAllocator extends ByteBlockPool.Allocator { private class ByteBlockAllocator extends ByteBlockPool.Allocator {
ArrayList freeByteBlocks = new ArrayList(); ArrayList<byte[]> freeByteBlocks = new ArrayList<byte[]>();
/* Allocate another byte[] from the shared pool */ /* Allocate another byte[] from the shared pool */
byte[] getByteBlock(boolean trackAllocations) { byte[] getByteBlock(boolean trackAllocations) {
@ -1231,7 +1229,7 @@ final class DocumentsWriter {
final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT; final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT;
final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1; final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1;
private ArrayList freeIntBlocks = new ArrayList(); private ArrayList<int[]> freeIntBlocks = new ArrayList<int[]>();
/* Allocate another int[] from the shared pool */ /* Allocate another int[] from the shared pool */
synchronized int[] getIntBlock(boolean trackAllocations) { synchronized int[] getIntBlock(boolean trackAllocations) {
@ -1280,7 +1278,7 @@ final class DocumentsWriter {
final static int MAX_TERM_LENGTH = CHAR_BLOCK_SIZE-1; final static int MAX_TERM_LENGTH = CHAR_BLOCK_SIZE-1;
private ArrayList freeCharBlocks = new ArrayList(); private ArrayList<char[]> freeCharBlocks = new ArrayList<char[]>();
/* Allocate another char[] from the shared pool */ /* Allocate another char[] from the shared pool */
synchronized char[] getCharBlock() { synchronized char[] getCharBlock() {

View File

@ -51,8 +51,8 @@ final class FieldInfos {
static final byte STORE_PAYLOADS = 0x20; static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40; static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
private final ArrayList byNumber = new ArrayList(); private final ArrayList<FieldInfo> byNumber = new ArrayList<FieldInfo>();
private final HashMap byName = new HashMap(); private final HashMap<String,FieldInfo> byName = new HashMap<String,FieldInfo>();
private int format; private int format;
FieldInfos() { } FieldInfos() { }
@ -111,10 +111,8 @@ final class FieldInfos {
/** Adds field info for a Document. */ /** Adds field info for a Document. */
synchronized public void add(Document doc) { synchronized public void add(Document doc) {
List fields = doc.getFields(); List<Fieldable> fields = doc.getFields();
Iterator fieldIterator = fields.iterator(); for (Fieldable field : fields) {
while (fieldIterator.hasNext()) {
Fieldable field = (Fieldable) fieldIterator.next();
add(field.name(), field.isIndexed(), field.isTermVectorStored(), field.isStorePositionWithTermVector(), add(field.name(), field.isIndexed(), field.isTermVectorStored(), field.isStorePositionWithTermVector(),
field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions()); field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
} }
@ -140,11 +138,10 @@ final class FieldInfos {
* @param storePositionWithTermVector true if positions should be stored. * @param storePositionWithTermVector true if positions should be stored.
* @param storeOffsetWithTermVector true if offsets should be stored * @param storeOffsetWithTermVector true if offsets should be stored
*/ */
synchronized public void addIndexed(Collection names, boolean storeTermVectors, boolean storePositionWithTermVector, synchronized public void addIndexed(Collection<String> names, boolean storeTermVectors, boolean storePositionWithTermVector,
boolean storeOffsetWithTermVector) { boolean storeOffsetWithTermVector) {
Iterator i = names.iterator(); for (String name : names) {
while (i.hasNext()) { add(name, true, storeTermVectors, storePositionWithTermVector, storeOffsetWithTermVector);
add((String)i.next(), true, storeTermVectors, storePositionWithTermVector, storeOffsetWithTermVector);
} }
} }
@ -156,10 +153,9 @@ final class FieldInfos {
* *
* @see #add(String, boolean) * @see #add(String, boolean)
*/ */
synchronized public void add(Collection names, boolean isIndexed) { synchronized public void add(Collection<String> names, boolean isIndexed) {
Iterator i = names.iterator(); for (String name : names) {
while (i.hasNext()) { add(name, isIndexed);
add((String)i.next(), isIndexed);
} }
} }

View File

@ -24,21 +24,21 @@ import java.util.*;
* This is not thread-safe. * This is not thread-safe.
*/ */
public class FieldSortedTermVectorMapper extends TermVectorMapper{ public class FieldSortedTermVectorMapper extends TermVectorMapper{
private Map fieldToTerms = new HashMap(); private Map<String,SortedSet<TermVectorEntry>> fieldToTerms = new HashMap<String,SortedSet<TermVectorEntry>>();
private SortedSet currentSet; private SortedSet<TermVectorEntry> currentSet;
private String currentField; private String currentField;
private Comparator comparator; private Comparator<TermVectorEntry> comparator;
/** /**
* *
* @param comparator A Comparator for sorting {@link TermVectorEntry}s * @param comparator A Comparator for sorting {@link TermVectorEntry}s
*/ */
public FieldSortedTermVectorMapper(Comparator comparator) { public FieldSortedTermVectorMapper(Comparator<TermVectorEntry> comparator) {
this(false, false, comparator); this(false, false, comparator);
} }
public FieldSortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator comparator) { public FieldSortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator<TermVectorEntry> comparator) {
super(ignoringPositions, ignoringOffsets); super(ignoringPositions, ignoringOffsets);
this.comparator = comparator; this.comparator = comparator;
} }
@ -49,7 +49,7 @@ public class FieldSortedTermVectorMapper extends TermVectorMapper{
} }
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) { public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
currentSet = new TreeSet(comparator); currentSet = new TreeSet<TermVectorEntry>(comparator);
currentField = field; currentField = field;
fieldToTerms.put(field, currentSet); fieldToTerms.put(field, currentSet);
} }
@ -59,12 +59,12 @@ public class FieldSortedTermVectorMapper extends TermVectorMapper{
* *
* @return A map between field names and {@link java.util.SortedSet}s per field. SortedSet entries are {@link TermVectorEntry} * @return A map between field names and {@link java.util.SortedSet}s per field. SortedSet entries are {@link TermVectorEntry}
*/ */
public Map getFieldToTerms() { public Map<String,SortedSet<TermVectorEntry>> getFieldToTerms() {
return fieldToTerms; return fieldToTerms;
} }
public Comparator getComparator() { public Comparator<TermVectorEntry> getComparator() {
return comparator; return comparator;
} }
} }

View File

@ -17,7 +17,7 @@ package org.apache.lucene.index;
*/ */
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.List;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable; import org.apache.lucene.document.Fieldable;
@ -215,17 +215,16 @@ final class FieldsWriter
indexStream.writeLong(fieldsStream.getFilePointer()); indexStream.writeLong(fieldsStream.getFilePointer());
int storedCount = 0; int storedCount = 0;
Iterator fieldIterator = doc.getFields().iterator(); List<Fieldable> fields = doc.getFields();
while (fieldIterator.hasNext()) { for (Fieldable field : fields) {
Fieldable field = (Fieldable) fieldIterator.next();
if (field.isStored()) if (field.isStored())
storedCount++; storedCount++;
} }
fieldsStream.writeVInt(storedCount); fieldsStream.writeVInt(storedCount);
fieldIterator = doc.getFields().iterator();
while (fieldIterator.hasNext()) {
Fieldable field = (Fieldable) fieldIterator.next(); for (Fieldable field : fields) {
if (field.isStored()) if (field.isStored())
writeField(fieldInfos.fieldInfo(field.name()), field); writeField(fieldInfos.fieldInfo(field.name()), field);
} }

View File

@ -52,7 +52,7 @@ public abstract class IndexCommit {
/** /**
* Returns all index files referenced by this commit point. * Returns all index files referenced by this commit point.
*/ */
public abstract Collection getFileNames() throws IOException; public abstract Collection<String> getFileNames() throws IOException;
/** /**
* Returns the {@link Directory} for the index. * Returns the {@link Directory} for the index.
@ -125,7 +125,7 @@ public abstract class IndexCommit {
/** Returns userData, previously passed to {@link /** Returns userData, previously passed to {@link
* IndexWriter#commit(Map)} for this commit. Map is * IndexWriter#commit(Map)} for this commit. Map is
* String -> String. */ * String -> String. */
public Map getUserData() throws IOException { public Map<String,String> getUserData() throws IOException {
throw new UnsupportedOperationException("This IndexCommit does not support this method."); throw new UnsupportedOperationException("This IndexCommit does not support this method.");
} }
} }

View File

@ -70,7 +70,7 @@ public interface IndexDeletionPolicy {
* {@link IndexCommit point-in-time commits}, * {@link IndexCommit point-in-time commits},
* sorted by age (the 0th one is the oldest commit). * sorted by age (the 0th one is the oldest commit).
*/ */
public void onInit(List commits) throws IOException; public void onInit(List<IndexCommit> commits) throws IOException;
/** /**
* <p>This is called each time the writer completed a commit. * <p>This is called each time the writer completed a commit.
@ -94,5 +94,5 @@ public interface IndexDeletionPolicy {
* @param commits List of {@link IndexCommit}, * @param commits List of {@link IndexCommit},
* sorted by age (the 0th one is the oldest commit). * sorted by age (the 0th one is the oldest commit).
*/ */
public void onCommit(List commits) throws IOException; public void onCommit(List<IndexCommit> commits) throws IOException;
} }

View File

@ -570,7 +570,7 @@ final class IndexFileDeleter {
final private static class CommitPoint extends IndexCommit implements Comparable<CommitPoint> { final private static class CommitPoint extends IndexCommit implements Comparable<CommitPoint> {
long gen; long gen;
Collection files; Collection<String> files;
String segmentsFileName; String segmentsFileName;
boolean deleted; boolean deleted;
Directory directory; Directory directory;
@ -602,7 +602,7 @@ final class IndexFileDeleter {
return segmentsFileName; return segmentsFileName;
} }
public Collection getFileNames() throws IOException { public Collection<String> getFileNames() throws IOException {
return files; return files;
} }

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index;
* limitations under the License. * limitations under the License.
*/ */
import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.io.IOException; import java.io.IOException;
@ -29,7 +30,7 @@ abstract class InvertedDocConsumer {
abstract void abort(); abstract void abort();
/** Flush a new segment */ /** Flush a new segment */
abstract void flush(Map threadsAndFields, SegmentWriteState state) throws IOException; abstract void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
/** Close doc stores */ /** Close doc stores */
abstract void closeDocStore(SegmentWriteState state) throws IOException; abstract void closeDocStore(SegmentWriteState state) throws IOException;

View File

@ -17,12 +17,13 @@ package org.apache.lucene.index;
* limitations under the License. * limitations under the License.
*/ */
import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.io.IOException; import java.io.IOException;
abstract class InvertedDocEndConsumer { abstract class InvertedDocEndConsumer {
abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread); abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread);
abstract void flush(Map threadsAndFields, SegmentWriteState state) throws IOException; abstract void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException;
abstract void closeDocStore(SegmentWriteState state) throws IOException; abstract void closeDocStore(SegmentWriteState state) throws IOException;
abstract void abort(); abstract void abort();
abstract void setFieldInfos(FieldInfos fieldInfos); abstract void setFieldInfos(FieldInfos fieldInfos);

View File

@ -31,7 +31,7 @@ public final class KeepOnlyLastCommitDeletionPolicy implements IndexDeletionPoli
/** /**
* Deletes all commits except the most recent one. * Deletes all commits except the most recent one.
*/ */
public void onInit(List commits) { public void onInit(List<IndexCommit> commits) {
// Note that commits.size() should normally be 1: // Note that commits.size() should normally be 1:
onCommit(commits); onCommit(commits);
} }
@ -39,7 +39,7 @@ public final class KeepOnlyLastCommitDeletionPolicy implements IndexDeletionPoli
/** /**
* Deletes all commits except the most recent one. * Deletes all commits except the most recent one.
*/ */
public void onCommit(List commits) { public void onCommit(List<IndexCommit> commits) {
// Note that commits.size() should normally be 2 (if not // Note that commits.size() should normally be 2 (if not
// called by onInit above): // called by onInit above):
int size = commits.size(); int size = commits.size();

View File

@ -175,7 +175,7 @@ public abstract class LogMergePolicy extends MergePolicy {
} }
} }
private boolean isOptimized(SegmentInfos infos, int maxNumSegments, Set segmentsToOptimize) throws IOException { private boolean isOptimized(SegmentInfos infos, int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
final int numSegments = infos.size(); final int numSegments = infos.size();
int numToOptimize = 0; int numToOptimize = 0;
SegmentInfo optimizeInfo = null; SegmentInfo optimizeInfo = null;
@ -212,7 +212,7 @@ public abstract class LogMergePolicy extends MergePolicy {
* (mergeFactor at a time) so the {@link MergeScheduler} * (mergeFactor at a time) so the {@link MergeScheduler}
* in use may make use of concurrency. */ * in use may make use of concurrency. */
public MergeSpecification findMergesForOptimize(SegmentInfos infos, public MergeSpecification findMergesForOptimize(SegmentInfos infos,
int maxNumSegments, Set segmentsToOptimize) throws IOException { int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
MergeSpecification spec; MergeSpecification spec;
assert maxNumSegments > 0; assert maxNumSegments > 0;

View File

@ -152,7 +152,7 @@ public abstract class MergePolicy {
* The subset of segments to be included in the primitive merge. * The subset of segments to be included in the primitive merge.
*/ */
public List merges = new ArrayList(); public List<OneMerge> merges = new ArrayList<OneMerge>();
public void add(OneMerge merge) { public void add(OneMerge merge) {
merges.add(merge); merges.add(merge);
@ -233,7 +233,7 @@ public abstract class MergePolicy {
* away. This may be a subset of all SegmentInfos. * away. This may be a subset of all SegmentInfos.
*/ */
public abstract MergeSpecification findMergesForOptimize( public abstract MergeSpecification findMergesForOptimize(
SegmentInfos segmentInfos, int maxSegmentCount, Set segmentsToOptimize) SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
throws CorruptIndexException, IOException; throws CorruptIndexException, IOException;
/** /**

View File

@ -36,7 +36,7 @@ public class MultiReader extends IndexReader implements Cloneable {
protected IndexReader[] subReaders; protected IndexReader[] subReaders;
private int[] starts; // 1st docno for each segment private int[] starts; // 1st docno for each segment
private boolean[] decrefOnClose; // remember which subreaders to decRef on close private boolean[] decrefOnClose; // remember which subreaders to decRef on close
private Map normsCache = new HashMap(); private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
private int maxDoc = 0; private int maxDoc = 0;
private int numDocs = -1; private int numDocs = -1;
private boolean hasDeletions = false; private boolean hasDeletions = false;
@ -284,7 +284,7 @@ public class MultiReader extends IndexReader implements Cloneable {
public synchronized byte[] norms(String field) throws IOException { public synchronized byte[] norms(String field) throws IOException {
ensureOpen(); ensureOpen();
byte[] bytes = (byte[])normsCache.get(field); byte[] bytes = normsCache.get(field);
if (bytes != null) if (bytes != null)
return bytes; // cache hit return bytes; // cache hit
if (!hasNorms(field)) if (!hasNorms(field))
@ -300,7 +300,7 @@ public class MultiReader extends IndexReader implements Cloneable {
public synchronized void norms(String field, byte[] result, int offset) public synchronized void norms(String field, byte[] result, int offset)
throws IOException { throws IOException {
ensureOpen(); ensureOpen();
byte[] bytes = (byte[])normsCache.get(field); byte[] bytes = normsCache.get(field);
for (int i = 0; i < subReaders.length; i++) // read from segments for (int i = 0; i < subReaders.length; i++) // read from segments
subReaders[i].norms(field, result, offset + starts[i]); subReaders[i].norms(field, result, offset + starts[i]);

View File

@ -21,7 +21,7 @@ import org.apache.lucene.util.PriorityQueue;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
@ -33,19 +33,17 @@ import java.util.List;
public class MultipleTermPositions implements TermPositions { public class MultipleTermPositions implements TermPositions {
private static final class TermPositionsQueue extends PriorityQueue<TermPositions> { private static final class TermPositionsQueue extends PriorityQueue<TermPositions> {
TermPositionsQueue(List termPositions) throws IOException { TermPositionsQueue(List<TermPositions> termPositions) throws IOException {
initialize(termPositions.size()); initialize(termPositions.size());
Iterator i = termPositions.iterator(); for (TermPositions tp : termPositions) {
while (i.hasNext()) {
TermPositions tp = (TermPositions) i.next();
if (tp.next()) if (tp.next())
add(tp); add(tp);
} }
} }
final TermPositions peek() { final TermPositions peek() {
return (TermPositions) top(); return top();
} }
public final boolean lessThan(TermPositions a, TermPositions b) { public final boolean lessThan(TermPositions a, TermPositions b) {
@ -102,7 +100,7 @@ public class MultipleTermPositions implements TermPositions {
* @exception IOException * @exception IOException
*/ */
public MultipleTermPositions(IndexReader indexReader, Term[] terms) throws IOException { public MultipleTermPositions(IndexReader indexReader, Term[] terms) throws IOException {
List termPositions = new LinkedList(); List<TermPositions> termPositions = new LinkedList<TermPositions>();
for (int i = 0; i < terms.length; i++) for (int i = 0; i < terms.length; i++)
termPositions.add(indexReader.termPositions(terms[i])); termPositions.add(indexReader.termPositions(terms[i]));

View File

@ -54,7 +54,7 @@ final class NormsWriter extends InvertedDocEndConsumer {
/** Produce _X.nrm if any document had a field with norms /** Produce _X.nrm if any document had a field with norms
* not disabled */ * not disabled */
public void flush(Map threadsAndFields, SegmentWriteState state) throws IOException { public void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
final Map byField = new HashMap(); final Map byField = new HashMap();

View File

@ -44,12 +44,12 @@ import java.util.*;
* undefined behavior</em>. * undefined behavior</em>.
*/ */
public class ParallelReader extends IndexReader { public class ParallelReader extends IndexReader {
private List readers = new ArrayList(); private List<IndexReader> readers = new ArrayList<IndexReader>();
private List decrefOnClose = new ArrayList(); // remember which subreaders to decRef on close private List<Boolean> decrefOnClose = new ArrayList<Boolean>(); // remember which subreaders to decRef on close
boolean incRefReaders = false; boolean incRefReaders = false;
private SortedMap fieldToReader = new TreeMap(); private SortedMap<String,IndexReader> fieldToReader = new TreeMap<String,IndexReader>();
private Map readerToFields = new HashMap(); private Map<IndexReader,Collection<String>> readerToFields = new HashMap<IndexReader,Collection<String>>();
private List storedFieldReaders = new ArrayList(); private List<IndexReader> storedFieldReaders = new ArrayList<IndexReader>();
private int maxDoc; private int maxDoc;
private int numDocs; private int numDocs;
@ -106,9 +106,7 @@ public class ParallelReader extends IndexReader {
Collection<String> fields = reader.getFieldNames(IndexReader.FieldOption.ALL); Collection<String> fields = reader.getFieldNames(IndexReader.FieldOption.ALL);
readerToFields.put(reader, fields); readerToFields.put(reader, fields);
Iterator i = fields.iterator(); for (final String field : fields) { // update fieldToReader map
while (i.hasNext()) { // update fieldToReader map
String field = (String)i.next();
if (fieldToReader.get(field) == null) if (fieldToReader.get(field) == null)
fieldToReader.put(field, reader); fieldToReader.put(field, reader);
} }
@ -158,13 +156,12 @@ public class ParallelReader extends IndexReader {
ensureOpen(); ensureOpen();
boolean reopened = false; boolean reopened = false;
List newReaders = new ArrayList(); List<IndexReader> newReaders = new ArrayList<IndexReader>();
boolean success = false; boolean success = false;
try { try {
for (int i = 0; i < readers.size(); i++) { for (final IndexReader oldReader : readers) {
IndexReader oldReader = (IndexReader) readers.get(i);
IndexReader newReader = null; IndexReader newReader = null;
if (doClone) { if (doClone) {
newReader = (IndexReader) oldReader.clone(); newReader = (IndexReader) oldReader.clone();
@ -182,7 +179,7 @@ public class ParallelReader extends IndexReader {
} finally { } finally {
if (!success && reopened) { if (!success && reopened) {
for (int i = 0; i < newReaders.size(); i++) { for (int i = 0; i < newReaders.size(); i++) {
IndexReader r = (IndexReader) newReaders.get(i); IndexReader r = newReaders.get(i);
if (r != readers.get(i)) { if (r != readers.get(i)) {
try { try {
r.close(); r.close();
@ -195,7 +192,7 @@ public class ParallelReader extends IndexReader {
} }
if (reopened) { if (reopened) {
List newDecrefOnClose = new ArrayList(); List<Boolean> newDecrefOnClose = new ArrayList<Boolean>();
ParallelReader pr = new ParallelReader(); ParallelReader pr = new ParallelReader();
for (int i = 0; i < readers.size(); i++) { for (int i = 0; i < readers.size(); i++) {
IndexReader oldReader = (IndexReader) readers.get(i); IndexReader oldReader = (IndexReader) readers.get(i);
@ -239,22 +236,22 @@ public class ParallelReader extends IndexReader {
public boolean isDeleted(int n) { public boolean isDeleted(int n) {
// Don't call ensureOpen() here (it could affect performance) // Don't call ensureOpen() here (it could affect performance)
if (readers.size() > 0) if (readers.size() > 0)
return ((IndexReader)readers.get(0)).isDeleted(n); return readers.get(0).isDeleted(n);
return false; return false;
} }
// delete in all readers // delete in all readers
protected void doDelete(int n) throws CorruptIndexException, IOException { protected void doDelete(int n) throws CorruptIndexException, IOException {
for (int i = 0; i < readers.size(); i++) { for (final IndexReader reader : readers) {
((IndexReader)readers.get(i)).deleteDocument(n); reader.deleteDocument(n);
} }
hasDeletions = true; hasDeletions = true;
} }
// undeleteAll in all readers // undeleteAll in all readers
protected void doUndeleteAll() throws CorruptIndexException, IOException { protected void doUndeleteAll() throws CorruptIndexException, IOException {
for (int i = 0; i < readers.size(); i++) { for (final IndexReader reader : readers) {
((IndexReader)readers.get(i)).undeleteAll(); reader.undeleteAll();
} }
hasDeletions = false; hasDeletions = false;
} }
@ -263,22 +260,21 @@ public class ParallelReader extends IndexReader {
public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException { public Document document(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
ensureOpen(); ensureOpen();
Document result = new Document(); Document result = new Document();
for (int i = 0; i < storedFieldReaders.size(); i++) { for (final IndexReader reader: storedFieldReaders) {
IndexReader reader = (IndexReader)storedFieldReaders.get(i);
boolean include = (fieldSelector==null); boolean include = (fieldSelector==null);
if (!include) { if (!include) {
Iterator it = ((Collection) readerToFields.get(reader)).iterator(); Collection<String> fields = readerToFields.get(reader);
while (it.hasNext()) for (final String field : fields)
if (fieldSelector.accept((String)it.next())!=FieldSelectorResult.NO_LOAD) { if (fieldSelector.accept(field) != FieldSelectorResult.NO_LOAD) {
include = true; include = true;
break; break;
} }
} }
if (include) { if (include) {
Iterator fieldIterator = reader.document(n, fieldSelector).getFields().iterator(); List<Fieldable> fields = reader.document(n, fieldSelector).getFields();
while (fieldIterator.hasNext()) { for (Fieldable field : fields) {
result.add((Fieldable)fieldIterator.next()); result.add(field);
} }
} }
} }
@ -288,12 +284,11 @@ public class ParallelReader extends IndexReader {
// get all vectors // get all vectors
public TermFreqVector[] getTermFreqVectors(int n) throws IOException { public TermFreqVector[] getTermFreqVectors(int n) throws IOException {
ensureOpen(); ensureOpen();
ArrayList results = new ArrayList(); ArrayList<TermFreqVector> results = new ArrayList<TermFreqVector>();
Iterator i = fieldToReader.entrySet().iterator(); for (final Map.Entry<String,IndexReader> e: fieldToReader.entrySet()) {
while (i.hasNext()) {
Map.Entry e = (Map.Entry)i.next(); String field = e.getKey();
String field = (String)e.getKey(); IndexReader reader = e.getValue();
IndexReader reader = (IndexReader)e.getValue();
TermFreqVector vector = reader.getTermFreqVector(n, field); TermFreqVector vector = reader.getTermFreqVector(n, field);
if (vector != null) if (vector != null)
results.add(vector); results.add(vector);
@ -305,14 +300,14 @@ public class ParallelReader extends IndexReader {
public TermFreqVector getTermFreqVector(int n, String field) public TermFreqVector getTermFreqVector(int n, String field)
throws IOException { throws IOException {
ensureOpen(); ensureOpen();
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
return reader==null ? null : reader.getTermFreqVector(n, field); return reader==null ? null : reader.getTermFreqVector(n, field);
} }
public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException { public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
ensureOpen(); ensureOpen();
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
if (reader != null) { if (reader != null) {
reader.getTermFreqVector(docNumber, field, mapper); reader.getTermFreqVector(docNumber, field, mapper);
} }
@ -321,11 +316,10 @@ public class ParallelReader extends IndexReader {
public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException { public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
ensureOpen(); ensureOpen();
Iterator i = fieldToReader.entrySet().iterator(); for (final Map.Entry<String,IndexReader> e : fieldToReader.entrySet()) {
while (i.hasNext()) {
Map.Entry e = (Map.Entry)i.next(); String field = e.getKey();
String field = (String)e.getKey(); IndexReader reader = e.getValue();
IndexReader reader = (IndexReader)e.getValue();
reader.getTermFreqVector(docNumber, field, mapper); reader.getTermFreqVector(docNumber, field, mapper);
} }
@ -333,27 +327,27 @@ public class ParallelReader extends IndexReader {
public boolean hasNorms(String field) throws IOException { public boolean hasNorms(String field) throws IOException {
ensureOpen(); ensureOpen();
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
return reader==null ? false : reader.hasNorms(field); return reader==null ? false : reader.hasNorms(field);
} }
public byte[] norms(String field) throws IOException { public byte[] norms(String field) throws IOException {
ensureOpen(); ensureOpen();
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
return reader==null ? null : reader.norms(field); return reader==null ? null : reader.norms(field);
} }
public void norms(String field, byte[] result, int offset) public void norms(String field, byte[] result, int offset)
throws IOException { throws IOException {
ensureOpen(); ensureOpen();
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
if (reader!=null) if (reader!=null)
reader.norms(field, result, offset); reader.norms(field, result, offset);
} }
protected void doSetNorm(int n, String field, byte value) protected void doSetNorm(int n, String field, byte value)
throws CorruptIndexException, IOException { throws CorruptIndexException, IOException {
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
if (reader!=null) if (reader!=null)
reader.doSetNorm(n, field, value); reader.doSetNorm(n, field, value);
} }
@ -370,7 +364,7 @@ public class ParallelReader extends IndexReader {
public int docFreq(Term term) throws IOException { public int docFreq(Term term) throws IOException {
ensureOpen(); ensureOpen();
IndexReader reader = ((IndexReader)fieldToReader.get(term.field())); IndexReader reader = fieldToReader.get(term.field());
return reader==null ? 0 : reader.docFreq(term); return reader==null ? 0 : reader.docFreq(term);
} }
@ -398,8 +392,8 @@ public class ParallelReader extends IndexReader {
* Checks recursively if all subreaders are up to date. * Checks recursively if all subreaders are up to date.
*/ */
public boolean isCurrent() throws CorruptIndexException, IOException { public boolean isCurrent() throws CorruptIndexException, IOException {
for (int i = 0; i < readers.size(); i++) { for (final IndexReader reader : readers) {
if (!((IndexReader)readers.get(i)).isCurrent()) { if (!reader.isCurrent()) {
return false; return false;
} }
} }
@ -412,8 +406,8 @@ public class ParallelReader extends IndexReader {
* Checks recursively if all subindexes are optimized * Checks recursively if all subindexes are optimized
*/ */
public boolean isOptimized() { public boolean isOptimized() {
for (int i = 0; i < readers.size(); i++) { for (final IndexReader reader : readers) {
if (!((IndexReader)readers.get(i)).isOptimized()) { if (!reader.isOptimized()) {
return false; return false;
} }
} }
@ -432,29 +426,28 @@ public class ParallelReader extends IndexReader {
// for testing // for testing
IndexReader[] getSubReaders() { IndexReader[] getSubReaders() {
return (IndexReader[]) readers.toArray(new IndexReader[readers.size()]); return readers.toArray(new IndexReader[readers.size()]);
} }
protected void doCommit(Map<String,String> commitUserData) throws IOException { protected void doCommit(Map<String,String> commitUserData) throws IOException {
for (int i = 0; i < readers.size(); i++) for (final IndexReader reader : readers)
((IndexReader)readers.get(i)).commit(commitUserData); reader.commit(commitUserData);
} }
protected synchronized void doClose() throws IOException { protected synchronized void doClose() throws IOException {
for (int i = 0; i < readers.size(); i++) { for (int i = 0; i < readers.size(); i++) {
if (((Boolean) decrefOnClose.get(i)).booleanValue()) { if (decrefOnClose.get(i).booleanValue()) {
((IndexReader)readers.get(i)).decRef(); readers.get(i).decRef();
} else { } else {
((IndexReader)readers.get(i)).close(); readers.get(i).close();
} }
} }
} }
public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) { public Collection<String> getFieldNames (IndexReader.FieldOption fieldNames) {
ensureOpen(); ensureOpen();
Set fieldSet = new HashSet(); Set<String> fieldSet = new HashSet<String>();
for (int i = 0; i < readers.size(); i++) { for (final IndexReader reader : readers) {
IndexReader reader = ((IndexReader)readers.get(i));
Collection<String> names = reader.getFieldNames(fieldNames); Collection<String> names = reader.getFieldNames(fieldNames);
fieldSet.addAll(names); fieldSet.addAll(names);
} }
@ -463,23 +456,23 @@ public class ParallelReader extends IndexReader {
private class ParallelTermEnum extends TermEnum { private class ParallelTermEnum extends TermEnum {
private String field; private String field;
private Iterator fieldIterator; private Iterator<String> fieldIterator;
private TermEnum termEnum; private TermEnum termEnum;
public ParallelTermEnum() throws IOException { public ParallelTermEnum() throws IOException {
try { try {
field = (String)fieldToReader.firstKey(); field = fieldToReader.firstKey();
} catch(NoSuchElementException e) { } catch(NoSuchElementException e) {
// No fields, so keep field == null, termEnum == null // No fields, so keep field == null, termEnum == null
return; return;
} }
if (field != null) if (field != null)
termEnum = ((IndexReader)fieldToReader.get(field)).terms(); termEnum = fieldToReader.get(field).terms();
} }
public ParallelTermEnum(Term term) throws IOException { public ParallelTermEnum(Term term) throws IOException {
field = term.field(); field = term.field();
IndexReader reader = ((IndexReader)fieldToReader.get(field)); IndexReader reader = fieldToReader.get(field);
if (reader!=null) if (reader!=null)
termEnum = reader.terms(term); termEnum = reader.terms(term);
} }
@ -500,8 +493,8 @@ public class ParallelReader extends IndexReader {
fieldIterator.next(); // Skip field to get next one fieldIterator.next(); // Skip field to get next one
} }
while (fieldIterator.hasNext()) { while (fieldIterator.hasNext()) {
field = (String) fieldIterator.next(); field = fieldIterator.next();
termEnum = ((IndexReader)fieldToReader.get(field)).terms(new Term(field)); termEnum = fieldToReader.get(field).terms(new Term(field));
Term term = termEnum.term(); Term term = termEnum.term();
if (term!=null && term.field()==field) if (term!=null && term.field()==field)
return true; return true;
@ -540,7 +533,7 @@ public class ParallelReader extends IndexReader {
public ParallelTermDocs() {} public ParallelTermDocs() {}
public ParallelTermDocs(Term term) throws IOException { public ParallelTermDocs(Term term) throws IOException {
if (term == null) if (term == null)
termDocs = readers.isEmpty() ? null : ((IndexReader)readers.get(0)).termDocs(null); termDocs = readers.isEmpty() ? null : readers.get(0).termDocs(null);
else else
seek(term); seek(term);
} }
@ -549,7 +542,7 @@ public class ParallelReader extends IndexReader {
public int freq() { return termDocs.freq(); } public int freq() { return termDocs.freq(); }
public void seek(Term term) throws IOException { public void seek(Term term) throws IOException {
IndexReader reader = ((IndexReader)fieldToReader.get(term.field())); IndexReader reader = fieldToReader.get(term.field());
termDocs = reader!=null ? reader.termDocs(term) : null; termDocs = reader!=null ? reader.termDocs(term) : null;
} }
@ -592,7 +585,7 @@ public class ParallelReader extends IndexReader {
public ParallelTermPositions(Term term) throws IOException { seek(term); } public ParallelTermPositions(Term term) throws IOException { seek(term); }
public void seek(Term term) throws IOException { public void seek(Term term) throws IOException {
IndexReader reader = ((IndexReader)fieldToReader.get(term.field())); IndexReader reader = fieldToReader.get(term.field());
termDocs = reader!=null ? reader.termPositions(term) : null; termDocs = reader!=null ? reader.termPositions(term) : null;
} }

View File

@ -27,13 +27,13 @@ import java.util.Map;
* This is not thread-safe. * This is not thread-safe.
*/ */
public class PositionBasedTermVectorMapper extends TermVectorMapper{ public class PositionBasedTermVectorMapper extends TermVectorMapper{
private Map/*<String, Map<Integer, TVPositionInfo>>*/ fieldToTerms; private Map<String, Map<Integer,TVPositionInfo>> fieldToTerms;
private String currentField; private String currentField;
/** /**
* A Map of Integer and TVPositionInfo * A Map of Integer and TVPositionInfo
*/ */
private Map/*<Integer, TVPositionInfo>*/ currentPositions; private Map<Integer,TVPositionInfo> currentPositions;
private boolean storeOffsets; private boolean storeOffsets;
@ -95,10 +95,10 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
{ {
//ignoring offsets //ignoring offsets
} }
fieldToTerms = new HashMap(numTerms); fieldToTerms = new HashMap<String,Map<Integer,TVPositionInfo>>(numTerms);
this.storeOffsets = storeOffsets; this.storeOffsets = storeOffsets;
currentField = field; currentField = field;
currentPositions = new HashMap(); currentPositions = new HashMap<Integer,TVPositionInfo>();
fieldToTerms.put(currentField, currentPositions); fieldToTerms.put(currentField, currentPositions);
} }
@ -107,7 +107,7 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
* *
* @return A map between field names and a Map. The sub-Map key is the position as the integer, the value is {@link org.apache.lucene.index.PositionBasedTermVectorMapper.TVPositionInfo}. * @return A map between field names and a Map. The sub-Map key is the position as the integer, the value is {@link org.apache.lucene.index.PositionBasedTermVectorMapper.TVPositionInfo}.
*/ */
public Map getFieldToTerms() { public Map<String, Map<Integer, TVPositionInfo>> getFieldToTerms() {
return fieldToTerms; return fieldToTerms;
} }
@ -116,17 +116,17 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
*/ */
public static class TVPositionInfo{ public static class TVPositionInfo{
private int position; private int position;
//a list of Strings
private List terms; private List<String> terms;
//A list of TermVectorOffsetInfo
private List offsets; private List<TermVectorOffsetInfo> offsets;
public TVPositionInfo(int position, boolean storeOffsets) { public TVPositionInfo(int position, boolean storeOffsets) {
this.position = position; this.position = position;
terms = new ArrayList(); terms = new ArrayList<String>();
if (storeOffsets) { if (storeOffsets) {
offsets = new ArrayList(); offsets = new ArrayList<TermVectorOffsetInfo>();
} }
} }
@ -150,15 +150,15 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
* Note, there may be multiple terms at the same position * Note, there may be multiple terms at the same position
* @return A List of Strings * @return A List of Strings
*/ */
public List getTerms() { public List<String> getTerms() {
return terms; return terms;
} }
/** /**
* Parallel list (to {@link #getTerms()}) of TermVectorOffsetInfo objects. There may be multiple entries since there may be multiple terms at a position * Parallel list (to {@link #getTerms()}) of TermVectorOffsetInfo objects. There may be multiple entries since there may be multiple terms at a position
* @return A List of TermVectorOffsetInfo objects, if offsets are store. * @return A List of TermVectorOffsetInfo objects, if offsets are stored.
*/ */
public List getOffsets() { public List<TermVectorOffsetInfo> getOffsets() {
return offsets; return offsets;
} }
} }

View File

@ -27,7 +27,7 @@ class ReadOnlyDirectoryReader extends DirectoryReader {
super(directory, sis, deletionPolicy, true, termInfosIndexDivisor); super(directory, sis, deletionPolicy, true, termInfosIndexDivisor);
} }
ReadOnlyDirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts, Map oldNormsCache, boolean doClone, ReadOnlyDirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts, Map<String,byte[]> oldNormsCache, boolean doClone,
int termInfosIndexDivisor) throws IOException { int termInfosIndexDivisor) throws IOException {
super(directory, infos, oldReaders, oldStarts, oldNormsCache, true, doClone, termInfosIndexDivisor); super(directory, infos, oldReaders, oldStarts, oldNormsCache, true, doClone, termInfosIndexDivisor);
} }

View File

@ -218,7 +218,7 @@ public final class SegmentInfo {
if (format <= SegmentInfos.FORMAT_DIAGNOSTICS) { if (format <= SegmentInfos.FORMAT_DIAGNOSTICS) {
diagnostics = input.readStringStringMap(); diagnostics = input.readStringStringMap();
} else { } else {
diagnostics = Collections.EMPTY_MAP; diagnostics = Collections.<String,String>emptyMap();
} }
} else { } else {
delGen = CHECK_DIR; delGen = CHECK_DIR;
@ -231,7 +231,7 @@ public final class SegmentInfo {
docStoreSegment = null; docStoreSegment = null;
delCount = -1; delCount = -1;
hasProx = true; hasProx = true;
diagnostics = Collections.EMPTY_MAP; diagnostics = Collections.<String,String>emptyMap();
} }
} }
@ -260,7 +260,7 @@ public final class SegmentInfo {
* this segment. */ * this segment. */
public long sizeInBytes() throws IOException { public long sizeInBytes() throws IOException {
if (sizeInBytes == -1) { if (sizeInBytes == -1) {
List files = files(); List<String> files = files();
final int size = files.size(); final int size = files.size();
sizeInBytes = 0; sizeInBytes = 0;
for(int i=0;i<size;i++) { for(int i=0;i<size;i++) {
@ -322,7 +322,7 @@ public final class SegmentInfo {
si.hasProx = hasProx; si.hasProx = hasProx;
si.preLockless = preLockless; si.preLockless = preLockless;
si.hasSingleNormFile = hasSingleNormFile; si.hasSingleNormFile = hasSingleNormFile;
si.diagnostics = new HashMap(diagnostics); si.diagnostics = new HashMap<String, String>(diagnostics);
if (normGen != null) { if (normGen != null) {
si.normGen = (long[]) normGen.clone(); si.normGen = (long[]) normGen.clone();
} }
@ -572,7 +572,7 @@ public final class SegmentInfo {
return hasProx; return hasProx;
} }
private void addIfExists(List files, String fileName) throws IOException { private void addIfExists(List<String> files, String fileName) throws IOException {
if (dir.fileExists(fileName)) if (dir.fileExists(fileName))
files.add(fileName); files.add(fileName);
} }

View File

@ -20,7 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator;
import java.util.List; import java.util.List;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
@ -50,7 +50,7 @@ final class SegmentMerger {
private String segment; private String segment;
private int termIndexInterval = IndexWriter.DEFAULT_TERM_INDEX_INTERVAL; private int termIndexInterval = IndexWriter.DEFAULT_TERM_INDEX_INTERVAL;
private List readers = new ArrayList(); private List<IndexReader> readers = new ArrayList<IndexReader>();
private FieldInfos fieldInfos; private FieldInfos fieldInfos;
private int mergedDocs; private int mergedDocs;
@ -164,18 +164,18 @@ final class SegmentMerger {
* @throws IOException * @throws IOException
*/ */
final void closeReaders() throws IOException { final void closeReaders() throws IOException {
for (Iterator iter = readers.iterator(); iter.hasNext();) { for (final IndexReader reader : readers) {
((IndexReader) iter.next()).close(); reader.close();
} }
} }
final List createCompoundFile(String fileName) final List<String> createCompoundFile(String fileName)
throws IOException { throws IOException {
CompoundFileWriter cfsWriter = CompoundFileWriter cfsWriter =
new CompoundFileWriter(directory, fileName, checkAbort); new CompoundFileWriter(directory, fileName, checkAbort);
List files = List<String> files =
new ArrayList(IndexFileNames.COMPOUND_EXTENSIONS.length + 1); new ArrayList<String>(IndexFileNames.COMPOUND_EXTENSIONS.length + 1);
// Basic files // Basic files
for (int i = 0; i < IndexFileNames.COMPOUND_EXTENSIONS.length; i++) { for (int i = 0; i < IndexFileNames.COMPOUND_EXTENSIONS.length; i++) {
@ -206,9 +206,8 @@ final class SegmentMerger {
} }
// Now merge all added files // Now merge all added files
Iterator it = files.iterator(); for (String file : files) {
while (it.hasNext()) { cfsWriter.addFile(file);
cfsWriter.addFile((String) it.next());
} }
// Perform the merge // Perform the merge
@ -218,13 +217,11 @@ final class SegmentMerger {
} }
private void addIndexed(IndexReader reader, FieldInfos fInfos, private void addIndexed(IndexReader reader, FieldInfos fInfos,
Collection names, boolean storeTermVectors, Collection<String> names, boolean storeTermVectors,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector, boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
boolean storePayloads, boolean omitTFAndPositions) boolean storePayloads, boolean omitTFAndPositions)
throws IOException { throws IOException {
Iterator i = names.iterator(); for (String field : names) {
while (i.hasNext()) {
String field = (String) i.next();
fInfos.add(field, true, storeTermVectors, fInfos.add(field, true, storeTermVectors,
storePositionWithTermVector, storeOffsetWithTermVector, !reader storePositionWithTermVector, storeOffsetWithTermVector, !reader
.hasNorms(field), storePayloads, omitTFAndPositions); .hasNorms(field), storePayloads, omitTFAndPositions);
@ -286,8 +283,7 @@ final class SegmentMerger {
fieldInfos = new FieldInfos(); // merge field names fieldInfos = new FieldInfos(); // merge field names
} }
for (Iterator iter = readers.iterator(); iter.hasNext();) { for (IndexReader reader : readers) {
IndexReader reader = (IndexReader) iter.next();
if (reader instanceof SegmentReader) { if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader; SegmentReader segmentReader = (SegmentReader) reader;
FieldInfos readerFieldInfos = segmentReader.fieldInfos(); FieldInfos readerFieldInfos = segmentReader.fieldInfos();
@ -322,8 +318,7 @@ final class SegmentMerger {
try { try {
int idx = 0; int idx = 0;
for (Iterator iter = readers.iterator(); iter.hasNext();) { for (IndexReader reader : readers) {
final IndexReader reader = (IndexReader) iter.next();
final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++]; final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
FieldsReader matchingFieldsReader = null; FieldsReader matchingFieldsReader = null;
if (matchingSegmentReader != null) { if (matchingSegmentReader != null) {
@ -359,8 +354,8 @@ final class SegmentMerger {
// If we are skipping the doc stores, that means there // If we are skipping the doc stores, that means there
// are no deletions in any of these segments, so we // are no deletions in any of these segments, so we
// just sum numDocs() of each segment to get total docCount // just sum numDocs() of each segment to get total docCount
for (Iterator iter = readers.iterator(); iter.hasNext();) { for (final IndexReader reader : readers) {
docCount += ((IndexReader) iter.next()).numDocs(); docCount += reader.numDocs();
} }
return docCount; return docCount;
@ -450,7 +445,7 @@ final class SegmentMerger {
try { try {
int idx = 0; int idx = 0;
for (Iterator iter = readers.iterator(); iter.hasNext();) { for (final IndexReader reader : readers) {
final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++]; final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
TermVectorsReader matchingVectorsReader = null; TermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) { if (matchingSegmentReader != null) {
@ -461,7 +456,6 @@ final class SegmentMerger {
matchingVectorsReader = vectorsReader; matchingVectorsReader = vectorsReader;
} }
} }
final IndexReader reader = (IndexReader) iter.next();
if (reader.hasDeletions()) { if (reader.hasDeletions()) {
copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader); copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader);
} else { } else {
@ -719,8 +713,7 @@ final class SegmentMerger {
output = directory.createOutput(segment + "." + IndexFileNames.NORMS_EXTENSION); output = directory.createOutput(segment + "." + IndexFileNames.NORMS_EXTENSION);
output.writeBytes(NORMS_HEADER,NORMS_HEADER.length); output.writeBytes(NORMS_HEADER,NORMS_HEADER.length);
} }
for (Iterator iter = readers.iterator(); iter.hasNext();) { for ( IndexReader reader : readers) {
IndexReader reader = (IndexReader) iter.next();
int maxDoc = reader.maxDoc(); int maxDoc = reader.maxDoc();
if (normBuffer == null || normBuffer.length < maxDoc) { if (normBuffer == null || normBuffer.length < maxDoc) {
// the buffer is too small for the current segment // the buffer is too small for the current segment

View File

@ -541,7 +541,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
} }
} }
Map norms = new HashMap(); Map<String,Norm> norms = new HashMap<String,Norm>();
/** The class which implements SegmentReader. */ /** The class which implements SegmentReader. */
// @deprecated (LUCENE-1677) // @deprecated (LUCENE-1677)
@ -762,7 +762,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
} }
} }
clone.norms = new HashMap(); clone.norms = new HashMap<String,Norm>();
// Clone norms // Clone norms
for (int i = 0; i < fieldNormsChanged.length; i++) { for (int i = 0; i < fieldNormsChanged.length; i++) {
@ -770,9 +770,9 @@ public class SegmentReader extends IndexReader implements Cloneable {
// Clone unchanged norms to the cloned reader // Clone unchanged norms to the cloned reader
if (doClone || !fieldNormsChanged[i]) { if (doClone || !fieldNormsChanged[i]) {
final String curField = core.fieldInfos.fieldInfo(i).name; final String curField = core.fieldInfos.fieldInfo(i).name;
Norm norm = (Norm) this.norms.get(curField); Norm norm = this.norms.get(curField);
if (norm != null) if (norm != null)
clone.norms.put(curField, norm.clone()); clone.norms.put(curField, (Norm) norm.clone());
} }
} }
@ -811,9 +811,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
if (normsDirty) { // re-write norms if (normsDirty) { // re-write norms
si.setNumFields(core.fieldInfos.size()); si.setNumFields(core.fieldInfos.size());
Iterator it = norms.values().iterator(); for (final Norm norm : norms.values()) {
while (it.hasNext()) {
Norm norm = (Norm) it.next();
if (norm.dirty) { if (norm.dirty) {
norm.reWrite(si); norm.reWrite(si);
} }
@ -839,9 +837,8 @@ public class SegmentReader extends IndexReader implements Cloneable {
deletedDocs = null; deletedDocs = null;
} }
Iterator it = norms.values().iterator(); for (final Norm norm : norms.values()) {
while (it.hasNext()) { norm.decRef();
((Norm) it.next()).decRef();
} }
if (core != null) { if (core != null) {
core.decRef(); core.decRef();
@ -901,8 +898,8 @@ public class SegmentReader extends IndexReader implements Cloneable {
} }
} }
List files() throws IOException { List<String> files() throws IOException {
return new ArrayList(si.files()); return new ArrayList<String>(si.files());
} }
public TermEnum terms() { public TermEnum terms() {
@ -1134,9 +1131,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
if (singleNormStream != null) { if (singleNormStream != null) {
return false; return false;
} }
Iterator it = norms.values().iterator(); for (final Norm norm : norms.values()) {
while (it.hasNext()) {
Norm norm = (Norm) it.next();
if (norm.refCount > 0) { if (norm.refCount > 0) {
return false; return false;
} }
@ -1146,8 +1141,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
// for testing only // for testing only
boolean normsClosed(String field) { boolean normsClosed(String field) {
Norm norm = (Norm) norms.get(field); return norms.get(field).refCount == 0;
return norm.refCount == 0;
} }
/** /**
@ -1263,9 +1257,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
rollbackDeletedDocsDirty = deletedDocsDirty; rollbackDeletedDocsDirty = deletedDocsDirty;
rollbackNormsDirty = normsDirty; rollbackNormsDirty = normsDirty;
rollbackPendingDeleteCount = pendingDeleteCount; rollbackPendingDeleteCount = pendingDeleteCount;
Iterator it = norms.values().iterator(); for (Norm norm : norms.values()) {
while (it.hasNext()) {
Norm norm = (Norm) it.next();
norm.rollbackDirty = norm.dirty; norm.rollbackDirty = norm.dirty;
} }
} }
@ -1275,9 +1267,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
deletedDocsDirty = rollbackDeletedDocsDirty; deletedDocsDirty = rollbackDeletedDocsDirty;
normsDirty = rollbackNormsDirty; normsDirty = rollbackNormsDirty;
pendingDeleteCount = rollbackPendingDeleteCount; pendingDeleteCount = rollbackPendingDeleteCount;
Iterator it = norms.values().iterator(); for (Norm norm : norms.values()) {
while (it.hasNext()) {
Norm norm = (Norm) it.next();
norm.dirty = norm.rollbackDirty; norm.dirty = norm.rollbackDirty;
} }
} }

View File

@ -52,14 +52,14 @@ public class SnapshotDeletionPolicy implements IndexDeletionPolicy {
this.primary = primary; this.primary = primary;
} }
public synchronized void onInit(List commits) throws IOException { public synchronized void onInit(List<IndexCommit> commits) throws IOException {
primary.onInit(wrapCommits(commits)); primary.onInit(wrapCommits(commits));
lastCommit = (IndexCommit) commits.get(commits.size()-1); lastCommit = commits.get(commits.size()-1);
} }
public synchronized void onCommit(List commits) throws IOException { public synchronized void onCommit(List<IndexCommit> commits) throws IOException {
primary.onCommit(wrapCommits(commits)); primary.onCommit(wrapCommits(commits));
lastCommit = (IndexCommit) commits.get(commits.size()-1); lastCommit = commits.get(commits.size()-1);
} }
/** Take a snapshot of the most recent commit to the /** Take a snapshot of the most recent commit to the
@ -95,7 +95,7 @@ public class SnapshotDeletionPolicy implements IndexDeletionPolicy {
public String getSegmentsFileName() { public String getSegmentsFileName() {
return cp.getSegmentsFileName(); return cp.getSegmentsFileName();
} }
public Collection getFileNames() throws IOException { public Collection<String> getFileNames() throws IOException {
return cp.getFileNames(); return cp.getFileNames();
} }
public Directory getDirectory() { public Directory getDirectory() {
@ -118,16 +118,16 @@ public class SnapshotDeletionPolicy implements IndexDeletionPolicy {
public long getGeneration() { public long getGeneration() {
return cp.getGeneration(); return cp.getGeneration();
} }
public Map getUserData() throws IOException { public Map<String,String> getUserData() throws IOException {
return cp.getUserData(); return cp.getUserData();
} }
} }
private List wrapCommits(List commits) { private List<IndexCommit> wrapCommits(List<IndexCommit> commits) {
final int count = commits.size(); final int count = commits.size();
List myCommits = new ArrayList(count); List<IndexCommit> myCommits = new ArrayList<IndexCommit>(count);
for(int i=0;i<count;i++) for(int i=0;i<count;i++)
myCommits.add(new MyCommitPoint((IndexCommit) commits.get(i))); myCommits.add(new MyCommitPoint(commits.get(i)));
return myCommits; return myCommits;
} }
} }

View File

@ -29,8 +29,8 @@ import java.util.*;
public class SortedTermVectorMapper extends TermVectorMapper{ public class SortedTermVectorMapper extends TermVectorMapper{
private SortedSet currentSet; private SortedSet<TermVectorEntry> currentSet;
private Map termToTVE = new HashMap(); private Map<String,TermVectorEntry> termToTVE = new HashMap<String,TermVectorEntry>();
private boolean storeOffsets; private boolean storeOffsets;
private boolean storePositions; private boolean storePositions;
/** /**
@ -42,14 +42,14 @@ public class SortedTermVectorMapper extends TermVectorMapper{
* *
* @param comparator A Comparator for sorting {@link TermVectorEntry}s * @param comparator A Comparator for sorting {@link TermVectorEntry}s
*/ */
public SortedTermVectorMapper(Comparator comparator) { public SortedTermVectorMapper(Comparator<TermVectorEntry> comparator) {
this(false, false, comparator); this(false, false, comparator);
} }
public SortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator comparator) { public SortedTermVectorMapper(boolean ignoringPositions, boolean ignoringOffsets, Comparator<TermVectorEntry> comparator) {
super(ignoringPositions, ignoringOffsets); super(ignoringPositions, ignoringOffsets);
currentSet = new TreeSet(comparator); currentSet = new TreeSet<TermVectorEntry>(comparator);
} }
/** /**
@ -121,7 +121,7 @@ public class SortedTermVectorMapper extends TermVectorMapper{
* *
* @return The SortedSet of {@link TermVectorEntry}. * @return The SortedSet of {@link TermVectorEntry}.
*/ */
public SortedSet getTermVectorEntrySet() public SortedSet<TermVectorEntry> getTermVectorEntrySet()
{ {
return currentSet; return currentSet;
} }

View File

@ -23,11 +23,9 @@ import java.util.Comparator;
* the term (case-sensitive) * the term (case-sensitive)
* *
**/ **/
public class TermVectorEntryFreqSortedComparator implements Comparator { public class TermVectorEntryFreqSortedComparator implements Comparator<TermVectorEntry> {
public int compare(Object object, Object object1) { public int compare(TermVectorEntry entry, TermVectorEntry entry1) {
int result = 0; int result = 0;
TermVectorEntry entry = (TermVectorEntry) object;
TermVectorEntry entry1 = (TermVectorEntry) object1;
result = entry1.getFrequency() - entry.getFrequency(); result = entry1.getFrequency() - entry.getFrequency();
if (result == 0) if (result == 0)
{ {

View File

@ -103,7 +103,7 @@ final class TermsHash extends InvertedDocConsumer {
nextTermsHash.closeDocStore(state); nextTermsHash.closeDocStore(state);
} }
synchronized void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException { synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
Map childThreadsAndFields = new HashMap(); Map childThreadsAndFields = new HashMap();
Map nextThreadsAndFields; Map nextThreadsAndFields;