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 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 abort();
abstract boolean freeRAM();

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
abstract class DocFieldConsumer {
@ -26,7 +27,7 @@ abstract class DocFieldConsumer {
/** Called when DocumentsWriter decides to create a new
* 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
* stores */

View File

@ -44,7 +44,7 @@ final class DocFieldConsumers extends DocFieldConsumer {
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 twoThreadsAndFields = new HashMap();

View File

@ -50,10 +50,11 @@ final class DocFieldProcessor extends DocConsumer {
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>>();
for ( DocFieldProcessorPerThread perThread : threads) {
for ( DocConsumerPerThread thread : threads) {
DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread;
childThreadsAndFields.put(perThread.consumer, perThread.fields());
perThread.trimFields(state);
}

View File

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

View File

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

View File

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

View File

@ -24,21 +24,21 @@ import java.util.*;
* This is not thread-safe.
*/
public class FieldSortedTermVectorMapper extends TermVectorMapper{
private Map fieldToTerms = new HashMap();
private SortedSet currentSet;
private Map<String,SortedSet<TermVectorEntry>> fieldToTerms = new HashMap<String,SortedSet<TermVectorEntry>>();
private SortedSet<TermVectorEntry> currentSet;
private String currentField;
private Comparator comparator;
private Comparator<TermVectorEntry> comparator;
/**
*
* @param comparator A Comparator for sorting {@link TermVectorEntry}s
*/
public FieldSortedTermVectorMapper(Comparator comparator) {
public FieldSortedTermVectorMapper(Comparator<TermVectorEntry> 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);
this.comparator = comparator;
}
@ -49,7 +49,7 @@ public class FieldSortedTermVectorMapper extends TermVectorMapper{
}
public void setExpectations(String field, int numTerms, boolean storeOffsets, boolean storePositions) {
currentSet = new TreeSet(comparator);
currentSet = new TreeSet<TermVectorEntry>(comparator);
currentField = field;
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}
*/
public Map getFieldToTerms() {
public Map<String,SortedSet<TermVectorEntry>> getFieldToTerms() {
return fieldToTerms;
}
public Comparator getComparator() {
public Comparator<TermVectorEntry> getComparator() {
return comparator;
}
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Fieldable;
@ -215,17 +215,16 @@ final class FieldsWriter
indexStream.writeLong(fieldsStream.getFilePointer());
int storedCount = 0;
Iterator fieldIterator = doc.getFields().iterator();
while (fieldIterator.hasNext()) {
Fieldable field = (Fieldable) fieldIterator.next();
List<Fieldable> fields = doc.getFields();
for (Fieldable field : fields) {
if (field.isStored())
storedCount++;
}
fieldsStream.writeVInt(storedCount);
fieldIterator = doc.getFields().iterator();
while (fieldIterator.hasNext()) {
Fieldable field = (Fieldable) fieldIterator.next();
for (Fieldable field : fields) {
if (field.isStored())
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.
*/
public abstract Collection getFileNames() throws IOException;
public abstract Collection<String> getFileNames() throws IOException;
/**
* Returns the {@link Directory} for the index.
@ -125,7 +125,7 @@ public abstract class IndexCommit {
/** Returns userData, previously passed to {@link
* IndexWriter#commit(Map)} for this commit. Map is
* String -> String. */
public Map getUserData() throws IOException {
public Map<String,String> getUserData() throws IOException {
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},
* 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.
@ -94,5 +94,5 @@ public interface IndexDeletionPolicy {
* @param commits List of {@link IndexCommit},
* 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> {
long gen;
Collection files;
Collection<String> files;
String segmentsFileName;
boolean deleted;
Directory directory;
@ -602,7 +602,7 @@ final class IndexFileDeleter {
return segmentsFileName;
}
public Collection getFileNames() throws IOException {
public Collection<String> getFileNames() throws IOException {
return files;
}

View File

@ -17,6 +17,7 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.util.Collection;
import java.util.Map;
import java.io.IOException;
@ -29,7 +30,7 @@ abstract class InvertedDocConsumer {
abstract void abort();
/** 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 */
abstract void closeDocStore(SegmentWriteState state) throws IOException;

View File

@ -17,12 +17,13 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.util.Collection;
import java.util.Map;
import java.io.IOException;
abstract class InvertedDocEndConsumer {
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 abort();
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.
*/
public void onInit(List commits) {
public void onInit(List<IndexCommit> commits) {
// Note that commits.size() should normally be 1:
onCommit(commits);
}
@ -39,7 +39,7 @@ public final class KeepOnlyLastCommitDeletionPolicy implements IndexDeletionPoli
/**
* 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
// called by onInit above):
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();
int numToOptimize = 0;
SegmentInfo optimizeInfo = null;
@ -212,7 +212,7 @@ public abstract class LogMergePolicy extends MergePolicy {
* (mergeFactor at a time) so the {@link MergeScheduler}
* in use may make use of concurrency. */
public MergeSpecification findMergesForOptimize(SegmentInfos infos,
int maxNumSegments, Set segmentsToOptimize) throws IOException {
int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
MergeSpecification spec;
assert maxNumSegments > 0;

View File

@ -152,7 +152,7 @@ public abstract class MergePolicy {
* 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) {
merges.add(merge);
@ -233,7 +233,7 @@ public abstract class MergePolicy {
* away. This may be a subset of all SegmentInfos.
*/
public abstract MergeSpecification findMergesForOptimize(
SegmentInfos segmentInfos, int maxSegmentCount, Set segmentsToOptimize)
SegmentInfos segmentInfos, int maxSegmentCount, Set<SegmentInfo> segmentsToOptimize)
throws CorruptIndexException, IOException;
/**

View File

@ -36,7 +36,7 @@ public class MultiReader extends IndexReader implements Cloneable {
protected IndexReader[] subReaders;
private int[] starts; // 1st docno for each segment
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 numDocs = -1;
private boolean hasDeletions = false;
@ -284,7 +284,7 @@ public class MultiReader extends IndexReader implements Cloneable {
public synchronized byte[] norms(String field) throws IOException {
ensureOpen();
byte[] bytes = (byte[])normsCache.get(field);
byte[] bytes = normsCache.get(field);
if (bytes != null)
return bytes; // cache hit
if (!hasNorms(field))
@ -300,7 +300,7 @@ public class MultiReader extends IndexReader implements Cloneable {
public synchronized void norms(String field, byte[] result, int offset)
throws IOException {
ensureOpen();
byte[] bytes = (byte[])normsCache.get(field);
byte[] bytes = normsCache.get(field);
for (int i = 0; i < subReaders.length; i++) // read from segments
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.util.Arrays;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@ -33,19 +33,17 @@ import java.util.List;
public class MultipleTermPositions implements TermPositions {
private static final class TermPositionsQueue extends PriorityQueue<TermPositions> {
TermPositionsQueue(List termPositions) throws IOException {
TermPositionsQueue(List<TermPositions> termPositions) throws IOException {
initialize(termPositions.size());
Iterator i = termPositions.iterator();
while (i.hasNext()) {
TermPositions tp = (TermPositions) i.next();
for (TermPositions tp : termPositions) {
if (tp.next())
add(tp);
}
}
final TermPositions peek() {
return (TermPositions) top();
return top();
}
public final boolean lessThan(TermPositions a, TermPositions b) {
@ -102,7 +100,7 @@ public class MultipleTermPositions implements TermPositions {
* @exception 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++)
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
* 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();

View File

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

View File

@ -27,13 +27,13 @@ import java.util.Map;
* This is not thread-safe.
*/
public class PositionBasedTermVectorMapper extends TermVectorMapper{
private Map/*<String, Map<Integer, TVPositionInfo>>*/ fieldToTerms;
private Map<String, Map<Integer,TVPositionInfo>> fieldToTerms;
private String currentField;
/**
* A Map of Integer and TVPositionInfo
*/
private Map/*<Integer, TVPositionInfo>*/ currentPositions;
private Map<Integer,TVPositionInfo> currentPositions;
private boolean storeOffsets;
@ -95,10 +95,10 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
{
//ignoring offsets
}
fieldToTerms = new HashMap(numTerms);
fieldToTerms = new HashMap<String,Map<Integer,TVPositionInfo>>(numTerms);
this.storeOffsets = storeOffsets;
currentField = field;
currentPositions = new HashMap();
currentPositions = new HashMap<Integer,TVPositionInfo>();
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}.
*/
public Map getFieldToTerms() {
public Map<String, Map<Integer, TVPositionInfo>> getFieldToTerms() {
return fieldToTerms;
}
@ -116,17 +116,17 @@ public class PositionBasedTermVectorMapper extends TermVectorMapper{
*/
public static class TVPositionInfo{
private int position;
//a list of Strings
private List terms;
//A list of TermVectorOffsetInfo
private List offsets;
private List<String> terms;
private List<TermVectorOffsetInfo> offsets;
public TVPositionInfo(int position, boolean storeOffsets) {
this.position = position;
terms = new ArrayList();
terms = new ArrayList<String>();
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
* @return A List of Strings
*/
public List getTerms() {
public List<String> getTerms() {
return terms;
}
/**
* 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;
}
}

View File

@ -27,7 +27,7 @@ class ReadOnlyDirectoryReader extends DirectoryReader {
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 {
super(directory, infos, oldReaders, oldStarts, oldNormsCache, true, doClone, termInfosIndexDivisor);
}

View File

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

View File

@ -20,7 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.document.Document;
@ -50,7 +50,7 @@ final class SegmentMerger {
private String segment;
private int termIndexInterval = IndexWriter.DEFAULT_TERM_INDEX_INTERVAL;
private List readers = new ArrayList();
private List<IndexReader> readers = new ArrayList<IndexReader>();
private FieldInfos fieldInfos;
private int mergedDocs;
@ -164,18 +164,18 @@ final class SegmentMerger {
* @throws IOException
*/
final void closeReaders() throws IOException {
for (Iterator iter = readers.iterator(); iter.hasNext();) {
((IndexReader) iter.next()).close();
for (final IndexReader reader : readers) {
reader.close();
}
}
final List createCompoundFile(String fileName)
final List<String> createCompoundFile(String fileName)
throws IOException {
CompoundFileWriter cfsWriter =
new CompoundFileWriter(directory, fileName, checkAbort);
List files =
new ArrayList(IndexFileNames.COMPOUND_EXTENSIONS.length + 1);
List<String> files =
new ArrayList<String>(IndexFileNames.COMPOUND_EXTENSIONS.length + 1);
// Basic files
for (int i = 0; i < IndexFileNames.COMPOUND_EXTENSIONS.length; i++) {
@ -206,9 +206,8 @@ final class SegmentMerger {
}
// Now merge all added files
Iterator it = files.iterator();
while (it.hasNext()) {
cfsWriter.addFile((String) it.next());
for (String file : files) {
cfsWriter.addFile(file);
}
// Perform the merge
@ -218,13 +217,11 @@ final class SegmentMerger {
}
private void addIndexed(IndexReader reader, FieldInfos fInfos,
Collection names, boolean storeTermVectors,
Collection<String> names, boolean storeTermVectors,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
boolean storePayloads, boolean omitTFAndPositions)
throws IOException {
Iterator i = names.iterator();
while (i.hasNext()) {
String field = (String) i.next();
for (String field : names) {
fInfos.add(field, true, storeTermVectors,
storePositionWithTermVector, storeOffsetWithTermVector, !reader
.hasNorms(field), storePayloads, omitTFAndPositions);
@ -286,8 +283,7 @@ final class SegmentMerger {
fieldInfos = new FieldInfos(); // merge field names
}
for (Iterator iter = readers.iterator(); iter.hasNext();) {
IndexReader reader = (IndexReader) iter.next();
for (IndexReader reader : readers) {
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
@ -322,8 +318,7 @@ final class SegmentMerger {
try {
int idx = 0;
for (Iterator iter = readers.iterator(); iter.hasNext();) {
final IndexReader reader = (IndexReader) iter.next();
for (IndexReader reader : readers) {
final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
FieldsReader matchingFieldsReader = null;
if (matchingSegmentReader != null) {
@ -359,8 +354,8 @@ final class SegmentMerger {
// If we are skipping the doc stores, that means there
// are no deletions in any of these segments, so we
// just sum numDocs() of each segment to get total docCount
for (Iterator iter = readers.iterator(); iter.hasNext();) {
docCount += ((IndexReader) iter.next()).numDocs();
for (final IndexReader reader : readers) {
docCount += reader.numDocs();
}
return docCount;
@ -450,7 +445,7 @@ final class SegmentMerger {
try {
int idx = 0;
for (Iterator iter = readers.iterator(); iter.hasNext();) {
for (final IndexReader reader : readers) {
final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
TermVectorsReader matchingVectorsReader = null;
if (matchingSegmentReader != null) {
@ -461,7 +456,6 @@ final class SegmentMerger {
matchingVectorsReader = vectorsReader;
}
}
final IndexReader reader = (IndexReader) iter.next();
if (reader.hasDeletions()) {
copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader);
} else {
@ -719,8 +713,7 @@ final class SegmentMerger {
output = directory.createOutput(segment + "." + IndexFileNames.NORMS_EXTENSION);
output.writeBytes(NORMS_HEADER,NORMS_HEADER.length);
}
for (Iterator iter = readers.iterator(); iter.hasNext();) {
IndexReader reader = (IndexReader) iter.next();
for ( IndexReader reader : readers) {
int maxDoc = reader.maxDoc();
if (normBuffer == null || normBuffer.length < maxDoc) {
// 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. */
// @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
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
if (doClone || !fieldNormsChanged[i]) {
final String curField = core.fieldInfos.fieldInfo(i).name;
Norm norm = (Norm) this.norms.get(curField);
Norm norm = this.norms.get(curField);
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
si.setNumFields(core.fieldInfos.size());
Iterator it = norms.values().iterator();
while (it.hasNext()) {
Norm norm = (Norm) it.next();
for (final Norm norm : norms.values()) {
if (norm.dirty) {
norm.reWrite(si);
}
@ -839,9 +837,8 @@ public class SegmentReader extends IndexReader implements Cloneable {
deletedDocs = null;
}
Iterator it = norms.values().iterator();
while (it.hasNext()) {
((Norm) it.next()).decRef();
for (final Norm norm : norms.values()) {
norm.decRef();
}
if (core != null) {
core.decRef();
@ -901,8 +898,8 @@ public class SegmentReader extends IndexReader implements Cloneable {
}
}
List files() throws IOException {
return new ArrayList(si.files());
List<String> files() throws IOException {
return new ArrayList<String>(si.files());
}
public TermEnum terms() {
@ -1134,9 +1131,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
if (singleNormStream != null) {
return false;
}
Iterator it = norms.values().iterator();
while (it.hasNext()) {
Norm norm = (Norm) it.next();
for (final Norm norm : norms.values()) {
if (norm.refCount > 0) {
return false;
}
@ -1146,8 +1141,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
// for testing only
boolean normsClosed(String field) {
Norm norm = (Norm) norms.get(field);
return norm.refCount == 0;
return norms.get(field).refCount == 0;
}
/**
@ -1263,9 +1257,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
rollbackDeletedDocsDirty = deletedDocsDirty;
rollbackNormsDirty = normsDirty;
rollbackPendingDeleteCount = pendingDeleteCount;
Iterator it = norms.values().iterator();
while (it.hasNext()) {
Norm norm = (Norm) it.next();
for (Norm norm : norms.values()) {
norm.rollbackDirty = norm.dirty;
}
}
@ -1275,9 +1267,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
deletedDocsDirty = rollbackDeletedDocsDirty;
normsDirty = rollbackNormsDirty;
pendingDeleteCount = rollbackPendingDeleteCount;
Iterator it = norms.values().iterator();
while (it.hasNext()) {
Norm norm = (Norm) it.next();
for (Norm norm : norms.values()) {
norm.dirty = norm.rollbackDirty;
}
}

View File

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

View File

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

View File

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

View File

@ -103,7 +103,7 @@ final class TermsHash extends InvertedDocConsumer {
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 nextThreadsAndFields;