mirror of https://github.com/apache/lucene.git
LUCENE-1257: More generified APIs and implementations (when will we be finished?). Thanks Kay Kay!
git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@826527 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
376873eba5
commit
504cd94c0b
|
@ -17,6 +17,7 @@ package org.apache.lucene.document;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -27,39 +28,39 @@ import java.util.Map;
|
|||
*/
|
||||
public class MapFieldSelector implements FieldSelector {
|
||||
|
||||
Map fieldSelections;
|
||||
Map<String,FieldSelectorResult> fieldSelections;
|
||||
|
||||
/** Create a a MapFieldSelector
|
||||
* @param fieldSelections maps from field names (String) to {@link FieldSelectorResult}s
|
||||
*/
|
||||
public MapFieldSelector(Map fieldSelections) {
|
||||
public MapFieldSelector(Map<String,FieldSelectorResult> fieldSelections) {
|
||||
this.fieldSelections = fieldSelections;
|
||||
}
|
||||
|
||||
/** Create a a MapFieldSelector
|
||||
* @param fields fields to LOAD. List of Strings. All other fields are NO_LOAD.
|
||||
*/
|
||||
public MapFieldSelector(List fields) {
|
||||
fieldSelections = new HashMap(fields.size()*5/3);
|
||||
for (int i=0; i<fields.size(); i++)
|
||||
fieldSelections.put(fields.get(i), FieldSelectorResult.LOAD);
|
||||
public MapFieldSelector(List<String> fields) {
|
||||
fieldSelections = new HashMap<String,FieldSelectorResult>(fields.size()*5/3);
|
||||
for (final String field : fields)
|
||||
fieldSelections.put(field, FieldSelectorResult.LOAD);
|
||||
}
|
||||
|
||||
/** Create a a MapFieldSelector
|
||||
* @param fields fields to LOAD. All other fields are NO_LOAD.
|
||||
*/
|
||||
public MapFieldSelector(String[] fields) {
|
||||
fieldSelections = new HashMap(fields.length*5/3);
|
||||
for (int i=0; i<fields.length; i++)
|
||||
fieldSelections.put(fields[i], FieldSelectorResult.LOAD);
|
||||
public MapFieldSelector(String... fields) {
|
||||
this(Arrays.asList(fields));
|
||||
}
|
||||
|
||||
|
||||
|
||||
/** Load field according to its associated value in fieldSelections
|
||||
* @param field a field name
|
||||
* @return the fieldSelections value that field maps to or NO_LOAD if none.
|
||||
*/
|
||||
public FieldSelectorResult accept(String field) {
|
||||
FieldSelectorResult selection = (FieldSelectorResult) fieldSelections.get(field);
|
||||
FieldSelectorResult selection = fieldSelections.get(field);
|
||||
return selection!=null ? selection : FieldSelectorResult.NO_LOAD;
|
||||
}
|
||||
|
||||
|
|
|
@ -23,18 +23,16 @@ import java.util.Set;
|
|||
**/
|
||||
public class SetBasedFieldSelector implements FieldSelector {
|
||||
|
||||
private Set fieldsToLoad;
|
||||
private Set lazyFieldsToLoad;
|
||||
private Set<String> fieldsToLoad;
|
||||
private Set<String> lazyFieldsToLoad;
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Pass in the Set of {@link Field} names to load and the Set of {@link Field} names to load lazily. If both are null, the
|
||||
* Document will not have any {@link Field} on it.
|
||||
* @param fieldsToLoad A Set of {@link String} field names to load. May be empty, but not null
|
||||
* @param lazyFieldsToLoad A Set of {@link String} field names to load lazily. May be empty, but not null
|
||||
*/
|
||||
public SetBasedFieldSelector(Set fieldsToLoad, Set lazyFieldsToLoad) {
|
||||
public SetBasedFieldSelector(Set<String> fieldsToLoad, Set<String> lazyFieldsToLoad) {
|
||||
this.fieldsToLoad = fieldsToLoad;
|
||||
this.lazyFieldsToLoad = lazyFieldsToLoad;
|
||||
}
|
||||
|
|
|
@ -17,8 +17,6 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -66,7 +64,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||
// under the same FieldInfo together, up into TermsHash*.
|
||||
// Other writers would presumably share alot of this...
|
||||
|
||||
public void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException {
|
||||
public void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
|
||||
|
||||
// Gather all FieldData's that have postings, across all
|
||||
// ThreadStates
|
||||
|
@ -279,7 +277,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||
|
||||
final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
|
||||
|
||||
void files(Collection files) {}
|
||||
void files(Collection<String> files) {}
|
||||
|
||||
static final class PostingList extends RawPostingList {
|
||||
int docFreq; // # times this term occurs in the current doc
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.io.FileNotFoundException;
|
|||
import java.io.PrintStream;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -72,12 +72,12 @@ final class IndexFileDeleter {
|
|||
/* Files that we tried to delete but failed (likely
|
||||
* because they are open and we are running on Windows),
|
||||
* so we will retry them again later: */
|
||||
private List deletable;
|
||||
private List<String> deletable;
|
||||
|
||||
/* Reference count for all files in the index.
|
||||
* Counts how many existing commits reference a file.
|
||||
* Maps String to RefCount (class below) instances: */
|
||||
private Map refCounts = new HashMap();
|
||||
**/
|
||||
private Map<String, RefCount> refCounts = new HashMap<String, RefCount>();
|
||||
|
||||
/* Holds all commits (segments_N) currently in the index.
|
||||
* This will have just 1 commit if you are using the
|
||||
|
@ -88,7 +88,7 @@ final class IndexFileDeleter {
|
|||
|
||||
/* Holds files we had incref'd from the previous
|
||||
* non-commit checkpoint: */
|
||||
private List lastFiles = new ArrayList();
|
||||
private List<Collection<String>> lastFiles = new ArrayList<Collection<String>>();
|
||||
|
||||
/* Commits that the IndexDeletionPolicy have decided to delete: */
|
||||
private List commitsToDelete = new ArrayList();
|
||||
|
@ -217,10 +217,9 @@ final class IndexFileDeleter {
|
|||
// Now delete anything with ref count at 0. These are
|
||||
// presumably abandoned files eg due to crash of
|
||||
// IndexWriter.
|
||||
Iterator it = refCounts.keySet().iterator();
|
||||
while(it.hasNext()) {
|
||||
String fileName = (String) it.next();
|
||||
RefCount rc = (RefCount) refCounts.get(fileName);
|
||||
for(Map.Entry<String, RefCount> entry : refCounts.entrySet() ) {
|
||||
RefCount rc = entry.getValue();
|
||||
final String fileName = entry.getKey();
|
||||
if (0 == rc.count) {
|
||||
if (infoStream != null) {
|
||||
message("init: removing unreferenced file \"" + fileName + "\"");
|
||||
|
@ -259,9 +258,8 @@ final class IndexFileDeleter {
|
|||
if (infoStream != null) {
|
||||
message("deleteCommits: now decRef commit \"" + commit.getSegmentsFileName() + "\"");
|
||||
}
|
||||
Iterator it = commit.files.iterator();
|
||||
while(it.hasNext()) {
|
||||
decRef((String) it.next());
|
||||
for (final String file : commit.files) {
|
||||
decRef(file);
|
||||
}
|
||||
}
|
||||
commitsToDelete.clear();
|
||||
|
@ -333,7 +331,7 @@ final class IndexFileDeleter {
|
|||
int size = lastFiles.size();
|
||||
if (size > 0) {
|
||||
for(int i=0;i<size;i++)
|
||||
decRef((Collection) lastFiles.get(i));
|
||||
decRef(lastFiles.get(i));
|
||||
lastFiles.clear();
|
||||
}
|
||||
|
||||
|
@ -342,13 +340,13 @@ final class IndexFileDeleter {
|
|||
|
||||
private void deletePendingFiles() throws IOException {
|
||||
if (deletable != null) {
|
||||
List oldDeletable = deletable;
|
||||
List<String> oldDeletable = deletable;
|
||||
deletable = null;
|
||||
int size = oldDeletable.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
if (infoStream != null)
|
||||
message("delete pending file " + oldDeletable.get(i));
|
||||
deleteFile((String) oldDeletable.get(i));
|
||||
deleteFile(oldDeletable.get(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -397,7 +395,7 @@ final class IndexFileDeleter {
|
|||
deleteCommits();
|
||||
} else {
|
||||
|
||||
final List docWriterFiles;
|
||||
final List<String> docWriterFiles;
|
||||
if (docWriter != null) {
|
||||
docWriterFiles = docWriter.openFiles();
|
||||
if (docWriterFiles != null)
|
||||
|
@ -412,7 +410,7 @@ final class IndexFileDeleter {
|
|||
int size = lastFiles.size();
|
||||
if (size > 0) {
|
||||
for(int i=0;i<size;i++)
|
||||
decRef((Collection) lastFiles.get(i));
|
||||
decRef(lastFiles.get(i));
|
||||
lastFiles.clear();
|
||||
}
|
||||
|
||||
|
@ -427,16 +425,14 @@ final class IndexFileDeleter {
|
|||
void incRef(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
|
||||
// If this is a commit point, also incRef the
|
||||
// segments_N file:
|
||||
Iterator it = segmentInfos.files(directory, isCommit).iterator();
|
||||
while(it.hasNext()) {
|
||||
incRef((String) it.next());
|
||||
for( final String fileName: segmentInfos.files(directory, isCommit) ) {
|
||||
incRef(fileName);
|
||||
}
|
||||
}
|
||||
|
||||
void incRef(List files) throws IOException {
|
||||
int size = files.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
incRef((String) files.get(i));
|
||||
void incRef(List<String> files) throws IOException {
|
||||
for(final String file : files) {
|
||||
incRef(file);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -448,10 +444,9 @@ final class IndexFileDeleter {
|
|||
rc.IncRef();
|
||||
}
|
||||
|
||||
void decRef(Collection files) throws IOException {
|
||||
Iterator it = files.iterator();
|
||||
while(it.hasNext()) {
|
||||
decRef((String) it.next());
|
||||
void decRef(Collection<String> files) throws IOException {
|
||||
for(final String file : files) {
|
||||
decRef(file);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -469,9 +464,8 @@ final class IndexFileDeleter {
|
|||
}
|
||||
|
||||
void decRef(SegmentInfos segmentInfos) throws IOException {
|
||||
Iterator it = segmentInfos.files(directory, false).iterator();
|
||||
while(it.hasNext()) {
|
||||
decRef((String) it.next());
|
||||
for (final String file : segmentInfos.files(directory, false)) {
|
||||
decRef(file);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -481,23 +475,20 @@ final class IndexFileDeleter {
|
|||
rc = new RefCount(fileName);
|
||||
refCounts.put(fileName, rc);
|
||||
} else {
|
||||
rc = (RefCount) refCounts.get(fileName);
|
||||
rc = refCounts.get(fileName);
|
||||
}
|
||||
return rc;
|
||||
}
|
||||
|
||||
void deleteFiles(List files) throws IOException {
|
||||
final int size = files.size();
|
||||
for(int i=0;i<size;i++)
|
||||
deleteFile((String) files.get(i));
|
||||
void deleteFiles(List<String> files) throws IOException {
|
||||
for(final String file: files)
|
||||
deleteFile(file);
|
||||
}
|
||||
|
||||
/** Deletes the specified files, but only if they are new
|
||||
* (have not yet been incref'd). */
|
||||
void deleteNewFiles(Collection files) throws IOException {
|
||||
final Iterator it = files.iterator();
|
||||
while(it.hasNext()) {
|
||||
final String fileName = (String) it.next();
|
||||
void deleteNewFiles(Collection<String> files) throws IOException {
|
||||
for (final String fileName: files) {
|
||||
if (!refCounts.containsKey(fileName))
|
||||
deleteFile(fileName);
|
||||
}
|
||||
|
@ -524,7 +515,7 @@ final class IndexFileDeleter {
|
|||
message("IndexFileDeleter: unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
|
||||
}
|
||||
if (deletable == null) {
|
||||
deletable = new ArrayList();
|
||||
deletable = new ArrayList<String>();
|
||||
}
|
||||
deletable.add(fileName); // add to deletable
|
||||
}
|
||||
|
@ -574,13 +565,13 @@ final class IndexFileDeleter {
|
|||
String segmentsFileName;
|
||||
boolean deleted;
|
||||
Directory directory;
|
||||
Collection commitsToDelete;
|
||||
Collection<IndexCommit> commitsToDelete;
|
||||
long version;
|
||||
long generation;
|
||||
final boolean isOptimized;
|
||||
final Map userData;
|
||||
final Map<String,String> userData;
|
||||
|
||||
public CommitPoint(Collection commitsToDelete, Directory directory, SegmentInfos segmentInfos) throws IOException {
|
||||
public CommitPoint(Collection<IndexCommit> commitsToDelete, Directory directory, SegmentInfos segmentInfos) throws IOException {
|
||||
this.directory = directory;
|
||||
this.commitsToDelete = commitsToDelete;
|
||||
userData = segmentInfos.getUserData();
|
||||
|
@ -618,7 +609,7 @@ final class IndexFileDeleter {
|
|||
return generation;
|
||||
}
|
||||
|
||||
public Map getUserData() {
|
||||
public Map<String,String> getUserData() {
|
||||
return userData;
|
||||
}
|
||||
|
||||
|
|
|
@ -27,16 +27,16 @@ import java.util.HashSet;
|
|||
public class IndexFileNameFilter implements FilenameFilter {
|
||||
|
||||
private static IndexFileNameFilter singleton = new IndexFileNameFilter();
|
||||
private HashSet extensions;
|
||||
private HashSet extensionsInCFS;
|
||||
private HashSet<String> extensions;
|
||||
private HashSet<String> extensionsInCFS;
|
||||
|
||||
// Prevent instantiation.
|
||||
private IndexFileNameFilter() {
|
||||
extensions = new HashSet();
|
||||
extensions = new HashSet<String>();
|
||||
for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS.length; i++) {
|
||||
extensions.add(IndexFileNames.INDEX_EXTENSIONS[i]);
|
||||
}
|
||||
extensionsInCFS = new HashSet();
|
||||
extensionsInCFS = new HashSet<String>();
|
||||
for (int i = 0; i < IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE.length; i++) {
|
||||
extensionsInCFS.add(IndexFileNames.INDEX_EXTENSIONS_IN_COMPOUND_FILE[i]);
|
||||
}
|
||||
|
|
|
@ -244,7 +244,7 @@ public class IndexWriter {
|
|||
private long lastCommitChangeCount; // last changeCount that was committed
|
||||
|
||||
private SegmentInfos rollbackSegmentInfos; // segmentInfos we will fallback to if the commit fails
|
||||
private HashMap rollbackSegments;
|
||||
private HashMap<SegmentInfo,Integer> rollbackSegments;
|
||||
|
||||
volatile SegmentInfos pendingCommit; // set when a commit is pending (after prepareCommit() & before commit())
|
||||
volatile long pendingCommitChangeCount;
|
||||
|
@ -257,7 +257,7 @@ public class IndexWriter {
|
|||
private DocumentsWriter docWriter;
|
||||
private IndexFileDeleter deleter;
|
||||
|
||||
private Set segmentsToOptimize = new HashSet(); // used by optimize to note those needing optimization
|
||||
private Set<SegmentInfo> segmentsToOptimize = new HashSet<SegmentInfo>(); // used by optimize to note those needing optimization
|
||||
|
||||
private Lock writeLock;
|
||||
|
||||
|
@ -268,13 +268,13 @@ public class IndexWriter {
|
|||
|
||||
// Holds all SegmentInfo instances currently involved in
|
||||
// merges
|
||||
private HashSet mergingSegments = new HashSet();
|
||||
private HashSet<SegmentInfo> mergingSegments = new HashSet<SegmentInfo>();
|
||||
|
||||
private MergePolicy mergePolicy = new LogByteSizeMergePolicy(this);
|
||||
private MergeScheduler mergeScheduler = new ConcurrentMergeScheduler();
|
||||
private LinkedList pendingMerges = new LinkedList();
|
||||
private Set runningMerges = new HashSet();
|
||||
private List mergeExceptions = new ArrayList();
|
||||
private LinkedList<MergePolicy.OneMerge> pendingMerges = new LinkedList<MergePolicy.OneMerge>();
|
||||
private Set<MergePolicy.OneMerge> runningMerges = new HashSet<MergePolicy.OneMerge>();
|
||||
private List<MergePolicy.OneMerge> mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
|
||||
private long mergeGen;
|
||||
private boolean stopMerges;
|
||||
|
||||
|
@ -402,23 +402,19 @@ public class IndexWriter {
|
|||
|
||||
class ReaderPool {
|
||||
|
||||
private final Map readerMap = new HashMap();
|
||||
private final Map<SegmentInfo,SegmentReader> readerMap = new HashMap<SegmentInfo,SegmentReader>();
|
||||
|
||||
/** Forcefully clear changes for the specified segments,
|
||||
* and remove from the pool. This is called on successful merge. */
|
||||
synchronized void clear(SegmentInfos infos) throws IOException {
|
||||
if (infos == null) {
|
||||
Iterator iter = readerMap.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
Map.Entry ent = (Map.Entry) iter.next();
|
||||
((SegmentReader) ent.getValue()).hasChanges = false;
|
||||
for (Map.Entry<SegmentInfo,SegmentReader> ent: readerMap.entrySet()) {
|
||||
ent.getValue().hasChanges = false;
|
||||
}
|
||||
} else {
|
||||
final int numSegments = infos.size();
|
||||
for(int i=0;i<numSegments;i++) {
|
||||
final SegmentInfo info = infos.info(i);
|
||||
for (final SegmentInfo info: infos) {
|
||||
if (readerMap.containsKey(info)) {
|
||||
((SegmentReader) readerMap.get(info)).hasChanges = false;
|
||||
readerMap.get(info).hasChanges = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -435,7 +431,7 @@ public class IndexWriter {
|
|||
public synchronized SegmentInfo mapToLive(SegmentInfo info) {
|
||||
int idx = segmentInfos.indexOf(info);
|
||||
if (idx != -1) {
|
||||
info = (SegmentInfo) segmentInfos.get(idx);
|
||||
info = segmentInfos.get(idx);
|
||||
}
|
||||
return info;
|
||||
}
|
||||
|
@ -497,11 +493,12 @@ public class IndexWriter {
|
|||
/** Remove all our references to readers, and commits
|
||||
* any pending changes. */
|
||||
synchronized void close() throws IOException {
|
||||
Iterator iter = readerMap.entrySet().iterator();
|
||||
Iterator<Map.Entry<SegmentInfo,SegmentReader>> iter = readerMap.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
Map.Entry ent = (Map.Entry) iter.next();
|
||||
|
||||
Map.Entry<SegmentInfo,SegmentReader> ent = iter.next();
|
||||
|
||||
SegmentReader sr = (SegmentReader) ent.getValue();
|
||||
SegmentReader sr = ent.getValue();
|
||||
if (sr.hasChanges) {
|
||||
assert infoIsLive(sr.getSegmentInfo());
|
||||
sr.startCommit();
|
||||
|
@ -531,11 +528,9 @@ public class IndexWriter {
|
|||
* @throws IOException
|
||||
*/
|
||||
synchronized void commit() throws IOException {
|
||||
Iterator iter = readerMap.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
Map.Entry ent = (Map.Entry) iter.next();
|
||||
for (Map.Entry<SegmentInfo,SegmentReader> ent : readerMap.entrySet()) {
|
||||
|
||||
SegmentReader sr = (SegmentReader) ent.getValue();
|
||||
SegmentReader sr = ent.getValue();
|
||||
if (sr.hasChanges) {
|
||||
assert infoIsLive(sr.getSegmentInfo());
|
||||
sr.startCommit();
|
||||
|
@ -1125,7 +1120,7 @@ public class IndexWriter {
|
|||
private synchronized void setRollbackSegmentInfos(SegmentInfos infos) {
|
||||
rollbackSegmentInfos = (SegmentInfos) infos.clone();
|
||||
assert !rollbackSegmentInfos.hasExternalSegments(directory);
|
||||
rollbackSegments = new HashMap();
|
||||
rollbackSegments = new HashMap<SegmentInfo,Integer>();
|
||||
final int size = rollbackSegmentInfos.size();
|
||||
for(int i=0;i<size;i++)
|
||||
rollbackSegments.put(rollbackSegmentInfos.info(i), Integer.valueOf(i));
|
||||
|
@ -1731,9 +1726,9 @@ public class IndexWriter {
|
|||
|
||||
try {
|
||||
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
|
||||
final Iterator it = docWriter.closedFiles().iterator();
|
||||
while(it.hasNext())
|
||||
cfsWriter.addFile((String) it.next());
|
||||
for (final String file : docWriter.closedFiles() ) {
|
||||
cfsWriter.addFile(file);
|
||||
}
|
||||
|
||||
// Perform the merge
|
||||
cfsWriter.close();
|
||||
|
@ -1923,7 +1918,7 @@ public class IndexWriter {
|
|||
// If docWriter has some aborted files that were
|
||||
// never incref'd, then we clean them up here
|
||||
if (docWriter != null) {
|
||||
final Collection files = docWriter.abortedFiles();
|
||||
final Collection<String> files = docWriter.abortedFiles();
|
||||
if (files != null)
|
||||
deleter.deleteNewFiles(files);
|
||||
}
|
||||
|
@ -2079,7 +2074,7 @@ public class IndexWriter {
|
|||
synchronized (this) {
|
||||
// If docWriter has some aborted files that were
|
||||
// never incref'd, then we clean them up here
|
||||
final Collection files = docWriter.abortedFiles();
|
||||
final Collection<String> files = docWriter.abortedFiles();
|
||||
if (files != null)
|
||||
deleter.deleteNewFiles(files);
|
||||
}
|
||||
|
@ -2254,23 +2249,19 @@ public class IndexWriter {
|
|||
|
||||
synchronized(this) {
|
||||
resetMergeExceptions();
|
||||
segmentsToOptimize = new HashSet();
|
||||
segmentsToOptimize = new HashSet<SegmentInfo>();
|
||||
final int numSegments = segmentInfos.size();
|
||||
for(int i=0;i<numSegments;i++)
|
||||
segmentsToOptimize.add(segmentInfos.info(i));
|
||||
|
||||
// Now mark all pending & running merges as optimize
|
||||
// merge:
|
||||
Iterator it = pendingMerges.iterator();
|
||||
while(it.hasNext()) {
|
||||
final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
|
||||
for(final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
merge.optimize = true;
|
||||
merge.maxNumSegmentsOptimize = maxNumSegments;
|
||||
}
|
||||
|
||||
it = runningMerges.iterator();
|
||||
while(it.hasNext()) {
|
||||
final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
|
||||
for ( final MergePolicy.OneMerge merge: runningMerges ) {
|
||||
merge.optimize = true;
|
||||
merge.maxNumSegmentsOptimize = maxNumSegments;
|
||||
}
|
||||
|
@ -2291,7 +2282,7 @@ public class IndexWriter {
|
|||
// threads to the current thread:
|
||||
final int size = mergeExceptions.size();
|
||||
for(int i=0;i<size;i++) {
|
||||
final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) mergeExceptions.get(0);
|
||||
final MergePolicy.OneMerge merge = mergeExceptions.get(i);
|
||||
if (merge.optimize) {
|
||||
IOException err = new IOException("background merge hit exception: " + merge.segString(directory));
|
||||
final Throwable t = merge.getException();
|
||||
|
@ -2324,16 +2315,16 @@ public class IndexWriter {
|
|||
/** Returns true if any merges in pendingMerges or
|
||||
* runningMerges are optimization merges. */
|
||||
private synchronized boolean optimizeMergesPending() {
|
||||
Iterator it = pendingMerges.iterator();
|
||||
while(it.hasNext())
|
||||
if (((MergePolicy.OneMerge) it.next()).optimize)
|
||||
for (final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
if (merge.optimize)
|
||||
return true;
|
||||
|
||||
it = runningMerges.iterator();
|
||||
while(it.hasNext())
|
||||
if (((MergePolicy.OneMerge) it.next()).optimize)
|
||||
}
|
||||
|
||||
for (final MergePolicy.OneMerge merge : runningMerges) {
|
||||
if (merge.optimize)
|
||||
return true;
|
||||
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -2513,7 +2504,7 @@ public class IndexWriter {
|
|||
if (pendingMerges.size() == 0)
|
||||
return null;
|
||||
else {
|
||||
Iterator it = pendingMerges.iterator();
|
||||
Iterator<MergePolicy.OneMerge> it = pendingMerges.iterator();
|
||||
while(it.hasNext()) {
|
||||
MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
|
||||
if (merge.isExternal) {
|
||||
|
@ -2810,9 +2801,7 @@ public class IndexWriter {
|
|||
stopMerges = true;
|
||||
|
||||
// Abort all pending & running merges:
|
||||
Iterator it = pendingMerges.iterator();
|
||||
while(it.hasNext()) {
|
||||
final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
|
||||
for (final MergePolicy.OneMerge merge : pendingMerges) {
|
||||
if (infoStream != null)
|
||||
message("now abort pending merge " + merge.segString(directory));
|
||||
merge.abort();
|
||||
|
@ -2820,9 +2809,7 @@ public class IndexWriter {
|
|||
}
|
||||
pendingMerges.clear();
|
||||
|
||||
it = runningMerges.iterator();
|
||||
while(it.hasNext()) {
|
||||
final MergePolicy.OneMerge merge = (MergePolicy.OneMerge) it.next();
|
||||
for (final MergePolicy.OneMerge merge : runningMerges) {
|
||||
if (infoStream != null)
|
||||
message("now abort running merge " + merge.segString(directory));
|
||||
merge.abort();
|
||||
|
@ -2918,12 +2905,12 @@ public class IndexWriter {
|
|||
}
|
||||
|
||||
private synchronized void resetMergeExceptions() {
|
||||
mergeExceptions = new ArrayList();
|
||||
mergeExceptions = new ArrayList<MergePolicy.OneMerge>();
|
||||
mergeGen++;
|
||||
}
|
||||
|
||||
private void noDupDirs(Directory... dirs) {
|
||||
HashSet dups = new HashSet();
|
||||
HashSet<Directory> dups = new HashSet<Directory>();
|
||||
for(int i=0;i<dirs.length;i++) {
|
||||
if (dups.contains(dirs[i]))
|
||||
throw new IllegalArgumentException("Directory " + dirs[i] + " appears more than once");
|
||||
|
@ -3251,7 +3238,7 @@ public class IndexWriter {
|
|||
|
||||
if (mergePolicy instanceof LogMergePolicy && getUseCompoundFile()) {
|
||||
|
||||
List files = null;
|
||||
List<String> files = null;
|
||||
|
||||
synchronized(this) {
|
||||
// Must incRef our files so that if another thread
|
||||
|
@ -3351,7 +3338,7 @@ public class IndexWriter {
|
|||
* only "stick" if there are actually changes in the
|
||||
* index to commit.
|
||||
*/
|
||||
public final void prepareCommit(Map commitUserData) throws CorruptIndexException, IOException {
|
||||
public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
|
||||
|
||||
if (hitOOM) {
|
||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
|
||||
|
@ -3416,7 +3403,7 @@ public class IndexWriter {
|
|||
* you should immediately close the writer. See <a
|
||||
* href="#OOME">above</a> for details.</p>
|
||||
*/
|
||||
public final void commit(Map commitUserData) throws CorruptIndexException, IOException {
|
||||
public final void commit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
|
||||
|
||||
ensureOpen();
|
||||
|
||||
|
@ -4115,7 +4102,7 @@ public class IndexWriter {
|
|||
false);
|
||||
|
||||
|
||||
Map details = new HashMap();
|
||||
Map<String,String> details = new HashMap<String,String>();
|
||||
details.put("optimize", merge.optimize+"");
|
||||
details.put("mergeFactor", end+"");
|
||||
details.put("mergeDocStores", mergeDocStores+"");
|
||||
|
@ -4132,8 +4119,8 @@ public class IndexWriter {
|
|||
setDiagnostics(info, source, null);
|
||||
}
|
||||
|
||||
private void setDiagnostics(SegmentInfo info, String source, Map details) {
|
||||
Map diagnostics = new HashMap();
|
||||
private void setDiagnostics(SegmentInfo info, String source, Map<String,String> details) {
|
||||
Map<String,String> diagnostics = new HashMap<String,String>();
|
||||
diagnostics.put("source", source);
|
||||
diagnostics.put("lucene.version", Constants.LUCENE_VERSION);
|
||||
diagnostics.put("os", Constants.OS_NAME+"");
|
||||
|
@ -4199,7 +4186,7 @@ public class IndexWriter {
|
|||
|
||||
boolean mergeDocStores = false;
|
||||
|
||||
final Set dss = new HashSet();
|
||||
final Set<String> dss = new HashSet<String>();
|
||||
|
||||
// This is try/finally to make sure merger's readers are
|
||||
// closed:
|
||||
|
@ -4463,12 +4450,12 @@ public class IndexWriter {
|
|||
}
|
||||
|
||||
// Files that have been sync'd already
|
||||
private HashSet synced = new HashSet();
|
||||
private HashSet<String> synced = new HashSet<String>();
|
||||
|
||||
// Files that are now being sync'd
|
||||
private HashSet syncing = new HashSet();
|
||||
private HashSet<String> syncing = new HashSet<String>();
|
||||
|
||||
private boolean startSync(String fileName, Collection pending) {
|
||||
private boolean startSync(String fileName, Collection<String> pending) {
|
||||
synchronized(synced) {
|
||||
if (!synced.contains(fileName)) {
|
||||
if (!syncing.contains(fileName)) {
|
||||
|
@ -4494,11 +4481,11 @@ public class IndexWriter {
|
|||
}
|
||||
|
||||
/** Blocks until all files in syncing are sync'd */
|
||||
private boolean waitForAllSynced(Collection syncing) throws IOException {
|
||||
private boolean waitForAllSynced(Collection<String> syncing) throws IOException {
|
||||
synchronized(synced) {
|
||||
Iterator it = syncing.iterator();
|
||||
Iterator<String> it = syncing.iterator();
|
||||
while(it.hasNext()) {
|
||||
final String fileName = (String) it.next();
|
||||
final String fileName = it.next();
|
||||
while(!synced.contains(fileName)) {
|
||||
if (!syncing.contains(fileName))
|
||||
// There was an error because a file that was
|
||||
|
@ -4541,7 +4528,7 @@ public class IndexWriter {
|
|||
* if it wasn't already. If that succeeds, then we
|
||||
* prepare a new segments_N file but do not fully commit
|
||||
* it. */
|
||||
private void startCommit(long sizeInBytes, Map commitUserData) throws IOException {
|
||||
private void startCommit(long sizeInBytes, Map<String,String> commitUserData) throws IOException {
|
||||
|
||||
assert testPoint("startStartCommit");
|
||||
|
||||
|
@ -4597,9 +4584,8 @@ public class IndexWriter {
|
|||
deleter.incRef(toSync, false);
|
||||
myChangeCount = changeCount;
|
||||
|
||||
Iterator it = toSync.files(directory, false).iterator();
|
||||
while(it.hasNext()) {
|
||||
String fileName = (String) it.next();
|
||||
Collection<String> files = toSync.files(directory, false);
|
||||
for(final String fileName: files) {
|
||||
assert directory.fileExists(fileName): "file " + fileName + " does not exist";
|
||||
}
|
||||
|
||||
|
@ -4617,11 +4603,11 @@ public class IndexWriter {
|
|||
// Loop until all files toSync references are sync'd:
|
||||
while(true) {
|
||||
|
||||
final Collection pending = new ArrayList();
|
||||
final Collection<String> pending = new ArrayList<String>();
|
||||
|
||||
Iterator it = toSync.files(directory, false).iterator();
|
||||
Iterator<String> it = toSync.files(directory, false).iterator();
|
||||
while(it.hasNext()) {
|
||||
final String fileName = (String) it.next();
|
||||
final String fileName = it.next();
|
||||
if (startSync(fileName, pending)) {
|
||||
boolean success = false;
|
||||
try {
|
||||
|
|
|
@ -46,7 +46,7 @@ final class NormsWriter extends InvertedDocEndConsumer {
|
|||
public void abort() {}
|
||||
|
||||
// We only write the _X.nrm file at flush
|
||||
void files(Collection files) {}
|
||||
void files(Collection<String> files) {}
|
||||
|
||||
void setFieldInfos(FieldInfos fieldInfos) {
|
||||
this.fieldInfos = fieldInfos;
|
||||
|
|
|
@ -112,7 +112,7 @@ public final class SegmentInfos extends Vector<SegmentInfo> {
|
|||
private static PrintStream infoStream;
|
||||
|
||||
public final SegmentInfo info(int i) {
|
||||
return (SegmentInfo) get(i);
|
||||
return get(i);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1031,7 +1031,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
|
||||
// can return null if norms aren't stored
|
||||
protected synchronized byte[] getNorms(String field) throws IOException {
|
||||
Norm norm = (Norm) norms.get(field);
|
||||
Norm norm = norms.get(field);
|
||||
if (norm == null) return null; // not indexed, or norms not stored
|
||||
return norm.bytes();
|
||||
}
|
||||
|
@ -1045,7 +1045,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
|
||||
protected void doSetNorm(int doc, String field, byte value)
|
||||
throws IOException {
|
||||
Norm norm = (Norm) norms.get(field);
|
||||
Norm norm = norms.get(field);
|
||||
if (norm == null) // not an indexed field
|
||||
return;
|
||||
|
||||
|
@ -1058,7 +1058,7 @@ public class SegmentReader extends IndexReader implements Cloneable {
|
|||
throws IOException {
|
||||
|
||||
ensureOpen();
|
||||
Norm norm = (Norm) norms.get(field);
|
||||
Norm norm = norms.get(field);
|
||||
if (norm == null) {
|
||||
Arrays.fill(bytes, offset, bytes.length, DefaultSimilarity.encodeNorm(1.0f));
|
||||
return;
|
||||
|
|
|
@ -52,7 +52,7 @@ final class TermInfosReader {
|
|||
SegmentTermEnum termEnum;
|
||||
|
||||
// Used for caching the least recently looked-up Terms
|
||||
Cache termInfoCache;
|
||||
Cache<Term,TermInfo> termInfoCache;
|
||||
}
|
||||
|
||||
TermInfosReader(Directory dir, String seg, FieldInfos fis, int readBufferSize, int indexDivisor)
|
||||
|
@ -143,7 +143,7 @@ final class TermInfosReader {
|
|||
resources = new ThreadResources();
|
||||
resources.termEnum = terms();
|
||||
// Cache does not have to be thread-safe, it is only used by one thread at the same time
|
||||
resources.termInfoCache = new SimpleLRUCache(DEFAULT_CACHE_SIZE);
|
||||
resources.termInfoCache = new SimpleLRUCache<Term,TermInfo>(DEFAULT_CACHE_SIZE);
|
||||
threadResources.set(resources);
|
||||
}
|
||||
return resources;
|
||||
|
@ -187,12 +187,12 @@ final class TermInfosReader {
|
|||
|
||||
TermInfo ti;
|
||||
ThreadResources resources = getThreadResources();
|
||||
Cache cache = null;
|
||||
Cache<Term,TermInfo> cache = null;
|
||||
|
||||
if (useCache) {
|
||||
cache = resources.termInfoCache;
|
||||
// check the cache first if the term was recently looked up
|
||||
ti = (TermInfo) cache.get(term);
|
||||
ti = cache.get(term);
|
||||
if (ti != null) {
|
||||
return ti;
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
final class TermVectorsTermsWriter extends TermsHashConsumer {
|
||||
|
@ -51,7 +51,7 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
postings[i] = new PostingList();
|
||||
}
|
||||
|
||||
synchronized void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException {
|
||||
synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
|
||||
|
||||
if (tvx != null) {
|
||||
|
||||
|
@ -65,12 +65,9 @@ final class TermVectorsTermsWriter extends TermsHashConsumer {
|
|||
tvf.flush();
|
||||
}
|
||||
|
||||
Iterator it = threadsAndFields.entrySet().iterator();
|
||||
while(it.hasNext()) {
|
||||
Map.Entry entry = (Map.Entry) it.next();
|
||||
Iterator it2 = ((Collection) entry.getValue()).iterator();
|
||||
while(it2.hasNext()) {
|
||||
TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) it2.next();
|
||||
for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
|
||||
for (final TermsHashConsumerPerField field : entry.getValue() ) {
|
||||
TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
|
||||
perField.termsHashPerField.reset();
|
||||
perField.shrinkHash();
|
||||
}
|
||||
|
|
|
@ -18,13 +18,14 @@ package org.apache.lucene.index;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
abstract class TermsHashConsumer {
|
||||
abstract int bytesPerPosting();
|
||||
abstract void createPostings(RawPostingList[] postings, int start, int count);
|
||||
abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
|
||||
abstract void flush(Map threadsAndFields, final SegmentWriteState state) throws IOException;
|
||||
abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
|
||||
abstract void abort();
|
||||
abstract void closeDocStore(SegmentWriteState state) throws IOException;
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search;
|
|||
import org.apache.lucene.index.IndexReader;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.BitSet;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.WeakHashMap;
|
||||
|
||||
|
@ -33,7 +33,7 @@ public class CachingSpanFilter extends SpanFilter {
|
|||
/**
|
||||
* A transient Filter cache.
|
||||
*/
|
||||
protected transient Map cache;
|
||||
protected transient Map<IndexReader,SpanFilterResult> cache;
|
||||
|
||||
/**
|
||||
* @param filter Filter to cache results of
|
||||
|
@ -50,11 +50,11 @@ public class CachingSpanFilter extends SpanFilter {
|
|||
private SpanFilterResult getCachedResult(IndexReader reader) throws IOException {
|
||||
SpanFilterResult result = null;
|
||||
if (cache == null) {
|
||||
cache = new WeakHashMap();
|
||||
cache = new WeakHashMap<IndexReader,SpanFilterResult>();
|
||||
}
|
||||
|
||||
synchronized (cache) { // check cache
|
||||
result = (SpanFilterResult) cache.get(reader);
|
||||
result = cache.get(reader);
|
||||
if (result == null) {
|
||||
result = filter.bitSpans(reader);
|
||||
cache.put(reader, result);
|
||||
|
|
|
@ -29,8 +29,8 @@ class ConjunctionScorer extends Scorer {
|
|||
private final float coord;
|
||||
private int lastDoc = -1;
|
||||
|
||||
public ConjunctionScorer(Similarity similarity, Collection scorers) throws IOException {
|
||||
this(similarity, (Scorer[]) scorers.toArray(new Scorer[scorers.size()]));
|
||||
public ConjunctionScorer(Similarity similarity, Collection<Scorer> scorers) throws IOException {
|
||||
this(similarity, scorers.toArray(new Scorer[scorers.size()]));
|
||||
}
|
||||
|
||||
public ConjunctionScorer(Similarity similarity, Scorer[] scorers) throws IOException {
|
||||
|
@ -52,9 +52,9 @@ class ConjunctionScorer extends Scorer {
|
|||
// it will already start off sorted (all scorers on same doc).
|
||||
|
||||
// note that this comparator is not consistent with equals!
|
||||
Arrays.sort(scorers, new Comparator() { // sort the array
|
||||
public int compare(Object o1, Object o2) {
|
||||
return ((Scorer) o1).docID() - ((Scorer) o2).docID();
|
||||
Arrays.sort(scorers, new Comparator<Scorer>() { // sort the array
|
||||
public int compare(Scorer o1, Scorer o2) {
|
||||
return o1.docID() - o2.docID();
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ import java.util.ArrayList;
|
|||
public class Explanation implements java.io.Serializable {
|
||||
private float value; // the value of this node
|
||||
private String description; // what it represents
|
||||
private ArrayList details; // sub-explanations
|
||||
private ArrayList<Explanation> details; // sub-explanations
|
||||
|
||||
public Explanation() {}
|
||||
|
||||
|
@ -71,13 +71,13 @@ public class Explanation implements java.io.Serializable {
|
|||
public Explanation[] getDetails() {
|
||||
if (details == null)
|
||||
return null;
|
||||
return (Explanation[])details.toArray(new Explanation[0]);
|
||||
return details.toArray(new Explanation[0]);
|
||||
}
|
||||
|
||||
/** Adds a sub-node to this explanation node. */
|
||||
public void addDetail(Explanation detail) {
|
||||
if (details == null)
|
||||
details = new ArrayList();
|
||||
details = new ArrayList<Explanation>();
|
||||
details.add(detail);
|
||||
}
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ public class FilterManager {
|
|||
protected static final long DEFAULT_CACHE_SLEEP_TIME = 1000 * 60 * 10;
|
||||
|
||||
/** The cache itself */
|
||||
protected Map cache;
|
||||
protected Map<Integer,FilterItem> cache;
|
||||
/** Maximum allowed cache size */
|
||||
protected int cacheCleanSize;
|
||||
/** Cache cleaning frequency */
|
||||
|
@ -65,7 +65,7 @@ public class FilterManager {
|
|||
* Sets up the FilterManager singleton.
|
||||
*/
|
||||
protected FilterManager() {
|
||||
cache = new HashMap();
|
||||
cache = new HashMap<Integer,FilterItem>();
|
||||
cacheCleanSize = DEFAULT_CACHE_CLEAN_SIZE; // Let the cache get to 100 items
|
||||
cleanSleepTime = DEFAULT_CACHE_SLEEP_TIME; // 10 minutes between cleanings
|
||||
|
||||
|
@ -103,7 +103,7 @@ public class FilterManager {
|
|||
public Filter getFilter(Filter filter) {
|
||||
synchronized(cache) {
|
||||
FilterItem fi = null;
|
||||
fi = (FilterItem)cache.get(Integer.valueOf(filter.hashCode()));
|
||||
fi = cache.get(Integer.valueOf(filter.hashCode()));
|
||||
if (fi != null) {
|
||||
fi.timestamp = new Date().getTime();
|
||||
return fi.filter;
|
||||
|
@ -146,14 +146,13 @@ public class FilterManager {
|
|||
protected class FilterCleaner implements Runnable {
|
||||
|
||||
private boolean running = true;
|
||||
private TreeSet sortedFilterItems;
|
||||
private TreeSet<Map.Entry<Integer,FilterItem>> sortedFilterItems;
|
||||
|
||||
public FilterCleaner() {
|
||||
sortedFilterItems = new TreeSet(new Comparator() {
|
||||
public int compare(Object a, Object b) {
|
||||
if( a instanceof Map.Entry && b instanceof Map.Entry) {
|
||||
FilterItem fia = (FilterItem) ((Map.Entry)a).getValue();
|
||||
FilterItem fib = (FilterItem) ((Map.Entry)b).getValue();
|
||||
sortedFilterItems = new TreeSet<Map.Entry<Integer,FilterItem>>(new Comparator<Map.Entry<Integer,FilterItem>>() {
|
||||
public int compare(Map.Entry<Integer,FilterItem> a, Map.Entry<Integer,FilterItem> b) {
|
||||
FilterItem fia = a.getValue();
|
||||
FilterItem fib = b.getValue();
|
||||
if ( fia.timestamp == fib.timestamp ) {
|
||||
return 0;
|
||||
}
|
||||
|
@ -163,9 +162,7 @@ public class FilterManager {
|
|||
}
|
||||
// larger timestamp last
|
||||
return 1;
|
||||
} else {
|
||||
throw new ClassCastException("Objects are not Map.Entry");
|
||||
}
|
||||
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -180,12 +177,12 @@ public class FilterManager {
|
|||
sortedFilterItems.clear();
|
||||
synchronized (cache) {
|
||||
sortedFilterItems.addAll(cache.entrySet());
|
||||
Iterator it = sortedFilterItems.iterator();
|
||||
Iterator<Map.Entry<Integer,FilterItem>> it = sortedFilterItems.iterator();
|
||||
int numToDelete = (int) ((cache.size() - cacheCleanSize) * 1.5);
|
||||
int counter = 0;
|
||||
// loop over the set and delete all of the cache entries not used in a while
|
||||
while (it.hasNext() && counter++ < numToDelete) {
|
||||
Map.Entry entry = (Map.Entry)it.next();
|
||||
Map.Entry<Integer,FilterItem> entry = it.next();
|
||||
cache.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -90,9 +90,9 @@ public class IndexSearcher extends Searcher {
|
|||
reader = r;
|
||||
this.closeReader = closeReader;
|
||||
|
||||
List subReadersList = new ArrayList();
|
||||
List<IndexReader> subReadersList = new ArrayList<IndexReader>();
|
||||
gatherSubReaders(subReadersList, reader);
|
||||
subReaders = (IndexReader[]) subReadersList.toArray(new IndexReader[subReadersList.size()]);
|
||||
subReaders = subReadersList.toArray(new IndexReader[subReadersList.size()]);
|
||||
docStarts = new int[subReaders.length];
|
||||
int maxDoc = 0;
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
|
@ -101,7 +101,7 @@ public class IndexSearcher extends Searcher {
|
|||
}
|
||||
}
|
||||
|
||||
protected void gatherSubReaders(List allSubReaders, IndexReader r) {
|
||||
protected void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader r) {
|
||||
ReaderUtil.gatherSubReaders(allSubReaders, r);
|
||||
}
|
||||
|
||||
|
|
|
@ -38,8 +38,8 @@ import org.apache.lucene.util.ToStringUtils;
|
|||
*/
|
||||
public class MultiPhraseQuery extends Query {
|
||||
private String field;
|
||||
private ArrayList termArrays = new ArrayList();
|
||||
private ArrayList positions = new ArrayList();
|
||||
private ArrayList<Term[]> termArrays = new ArrayList<Term[]>();
|
||||
private ArrayList<Integer> positions = new ArrayList<Integer>();
|
||||
|
||||
private int slop = 0;
|
||||
|
||||
|
@ -95,10 +95,10 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns a List<Term[]> of the terms in the multiphrase.
|
||||
* Returns a List of the terms in the multiphrase.
|
||||
* Do not modify the List or its contents.
|
||||
*/
|
||||
public List getTermArrays() {
|
||||
public List<Term[]> getTermArrays() {
|
||||
return Collections.unmodifiableList(termArrays);
|
||||
}
|
||||
|
||||
|
@ -114,10 +114,9 @@ public class MultiPhraseQuery extends Query {
|
|||
|
||||
// inherit javadoc
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
for (Iterator iter = termArrays.iterator(); iter.hasNext();) {
|
||||
Term[] arr = (Term[])iter.next();
|
||||
for (int i=0; i<arr.length; i++) {
|
||||
terms.add(arr[i]);
|
||||
for (final Term[] arr : termArrays) {
|
||||
for (final Term term: arr) {
|
||||
terms.add(term);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -135,11 +134,9 @@ public class MultiPhraseQuery extends Query {
|
|||
this.similarity = getSimilarity(searcher);
|
||||
|
||||
// compute idf
|
||||
Iterator i = termArrays.iterator();
|
||||
while (i.hasNext()) {
|
||||
Term[] terms = (Term[])i.next();
|
||||
for (int j=0; j<terms.length; j++) {
|
||||
idf += getSimilarity(searcher).idf(terms[j], searcher);
|
||||
for(final Term[] terms: termArrays) {
|
||||
for (Term term: terms) {
|
||||
idf += getSimilarity(searcher).idf(term, searcher);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -278,9 +275,9 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
buffer.append("\"");
|
||||
Iterator i = termArrays.iterator();
|
||||
Iterator<Term[]> i = termArrays.iterator();
|
||||
while (i.hasNext()) {
|
||||
Term[] terms = (Term[])i.next();
|
||||
Term[] terms = i.next();
|
||||
if (terms.length > 1) {
|
||||
buffer.append("(");
|
||||
for (int j = 0; j < terms.length; j++) {
|
||||
|
@ -330,9 +327,7 @@ public class MultiPhraseQuery extends Query {
|
|||
// Breakout calculation of the termArrays hashcode
|
||||
private int termArraysHashCode() {
|
||||
int hashCode = 1;
|
||||
Iterator iterator = termArrays.iterator();
|
||||
while (iterator.hasNext()) {
|
||||
Term[] termArray = (Term[]) iterator.next();
|
||||
for (final Term[] termArray: termArrays) {
|
||||
hashCode = 31 * hashCode
|
||||
+ (termArray == null ? 0 : arraysHashCode(termArray));
|
||||
}
|
||||
|
@ -354,15 +349,15 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
// Breakout calculation of the termArrays equals
|
||||
private boolean termArraysEquals(List termArrays1, List termArrays2) {
|
||||
private boolean termArraysEquals(List<Term[]> termArrays1, List<Term[]> termArrays2) {
|
||||
if (termArrays1.size() != termArrays2.size()) {
|
||||
return false;
|
||||
}
|
||||
ListIterator iterator1 = termArrays1.listIterator();
|
||||
ListIterator iterator2 = termArrays2.listIterator();
|
||||
ListIterator<Term[]> iterator1 = termArrays1.listIterator();
|
||||
ListIterator<Term[]> iterator2 = termArrays2.listIterator();
|
||||
while (iterator1.hasNext()) {
|
||||
Term[] termArray1 = (Term[]) iterator1.next();
|
||||
Term[] termArray2 = (Term[]) iterator2.next();
|
||||
Term[] termArray1 = iterator1.next();
|
||||
Term[] termArray2 = iterator2.next();
|
||||
if (!(termArray1 == null ? termArray2 == null : Arrays.equals(termArray1,
|
||||
termArray2))) {
|
||||
return false;
|
||||
|
|
|
@ -43,10 +43,10 @@ public class MultiSearcher extends Searcher {
|
|||
* initialize Weights.
|
||||
*/
|
||||
private static class CachedDfSource extends Searcher {
|
||||
private Map dfMap; // Map from Terms to corresponding doc freqs
|
||||
private Map<Term,Integer> dfMap; // Map from Terms to corresponding doc freqs
|
||||
private int maxDoc; // document count
|
||||
|
||||
public CachedDfSource(Map dfMap, int maxDoc, Similarity similarity) {
|
||||
public CachedDfSource(Map<Term,Integer> dfMap, int maxDoc, Similarity similarity) {
|
||||
this.dfMap = dfMap;
|
||||
this.maxDoc = maxDoc;
|
||||
setSimilarity(similarity);
|
||||
|
@ -55,7 +55,7 @@ public class MultiSearcher extends Searcher {
|
|||
public int docFreq(Term term) {
|
||||
int df;
|
||||
try {
|
||||
df = ((Integer) dfMap.get(term)).intValue();
|
||||
df = dfMap.get(term).intValue();
|
||||
} catch (NullPointerException e) {
|
||||
throw new IllegalArgumentException("df for term " + term.text()
|
||||
+ " not available");
|
||||
|
@ -305,7 +305,7 @@ public class MultiSearcher extends Searcher {
|
|||
Query rewrittenQuery = rewrite(original);
|
||||
|
||||
// step 2
|
||||
Set terms = new HashSet();
|
||||
Set<Term> terms = new HashSet<Term>();
|
||||
rewrittenQuery.extractTerms(terms);
|
||||
|
||||
// step3
|
||||
|
@ -319,7 +319,7 @@ public class MultiSearcher extends Searcher {
|
|||
}
|
||||
}
|
||||
|
||||
HashMap dfMap = new HashMap();
|
||||
HashMap<Term,Integer> dfMap = new HashMap<Term,Integer>();
|
||||
for(int i=0; i<allTermsArray.length; i++) {
|
||||
dfMap.put(allTermsArray[i], Integer.valueOf(aggregatedDfs[i]));
|
||||
}
|
||||
|
|
|
@ -21,11 +21,11 @@ import java.io.IOException;
|
|||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.util.ToStringUtils;
|
||||
|
||||
import org.apache.lucene.queryParser.QueryParser; // for javadoc
|
||||
|
||||
/**
|
||||
|
@ -217,7 +217,7 @@ public abstract class MultiTermQuery extends Query {
|
|||
// exhaust the enum before hitting either of the
|
||||
// cutoffs, we use ConstantBooleanQueryRewrite; else,
|
||||
// ConstantFilterRewrite:
|
||||
final Collection pendingTerms = new ArrayList();
|
||||
final Collection<Term> pendingTerms = new ArrayList<Term>();
|
||||
final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc());
|
||||
final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff);
|
||||
int docVisitCount = 0;
|
||||
|
@ -244,10 +244,9 @@ public abstract class MultiTermQuery extends Query {
|
|||
// Enumeration is done, and we hit a small
|
||||
// enough number of terms & docs -- just make a
|
||||
// BooleanQuery, now
|
||||
Iterator it = pendingTerms.iterator();
|
||||
BooleanQuery bq = new BooleanQuery(true);
|
||||
while(it.hasNext()) {
|
||||
TermQuery tq = new TermQuery((Term) it.next());
|
||||
for (final Term term: pendingTerms) {
|
||||
TermQuery tq = new TermQuery(term);
|
||||
bq.add(tq, BooleanClause.Occur.SHOULD);
|
||||
}
|
||||
// Strip scores
|
||||
|
|
|
@ -34,8 +34,8 @@ import org.apache.lucene.util.ToStringUtils;
|
|||
*/
|
||||
public class PhraseQuery extends Query {
|
||||
private String field;
|
||||
private ArrayList terms = new ArrayList(4);
|
||||
private ArrayList positions = new ArrayList(4);
|
||||
private ArrayList<Term> terms = new ArrayList<Term>(4);
|
||||
private ArrayList<Integer> positions = new ArrayList<Integer>(4);
|
||||
private int maxPosition = 0;
|
||||
private int slop = 0;
|
||||
|
||||
|
@ -67,7 +67,7 @@ public class PhraseQuery extends Query {
|
|||
public void add(Term term) {
|
||||
int position = 0;
|
||||
if(positions.size() > 0)
|
||||
position = ((Integer) positions.get(positions.size()-1)).intValue() + 1;
|
||||
position = positions.get(positions.size()-1).intValue() + 1;
|
||||
|
||||
add(term, position);
|
||||
}
|
||||
|
@ -94,7 +94,7 @@ public class PhraseQuery extends Query {
|
|||
|
||||
/** Returns the set of terms in this phrase. */
|
||||
public Term[] getTerms() {
|
||||
return (Term[])terms.toArray(new Term[0]);
|
||||
return terms.toArray(new Term[0]);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -103,7 +103,7 @@ public class PhraseQuery extends Query {
|
|||
public int[] getPositions() {
|
||||
int[] result = new int[positions.size()];
|
||||
for(int i = 0; i < positions.size(); i++)
|
||||
result[i] = ((Integer) positions.get(i)).intValue();
|
||||
result[i] = positions.get(i).intValue();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -145,7 +145,7 @@ public class PhraseQuery extends Query {
|
|||
|
||||
TermPositions[] tps = new TermPositions[terms.size()];
|
||||
for (int i = 0; i < terms.size(); i++) {
|
||||
TermPositions p = reader.termPositions((Term)terms.get(i));
|
||||
TermPositions p = reader.termPositions(terms.get(i));
|
||||
if (p == null)
|
||||
return null;
|
||||
tps[i] = p;
|
||||
|
@ -176,7 +176,7 @@ public class PhraseQuery extends Query {
|
|||
query.append(" ");
|
||||
}
|
||||
|
||||
Term term = (Term)terms.get(i);
|
||||
Term term = terms.get(i);
|
||||
|
||||
query.append(term.text());
|
||||
}
|
||||
|
@ -242,7 +242,7 @@ public class PhraseQuery extends Query {
|
|||
|
||||
public Weight createWeight(Searcher searcher) throws IOException {
|
||||
if (terms.size() == 1) { // optimize one-term case
|
||||
Term term = (Term)terms.get(0);
|
||||
Term term = terms.get(0);
|
||||
Query termQuery = new TermQuery(term);
|
||||
termQuery.setBoost(getBoost());
|
||||
return termQuery.createWeight(searcher);
|
||||
|
@ -268,12 +268,12 @@ public class PhraseQuery extends Query {
|
|||
buffer.append("\"");
|
||||
String[] pieces = new String[maxPosition + 1];
|
||||
for (int i = 0; i < terms.size(); i++) {
|
||||
int pos = ((Integer)positions.get(i)).intValue();
|
||||
int pos = positions.get(i).intValue();
|
||||
String s = pieces[pos];
|
||||
if (s == null) {
|
||||
s = ((Term)terms.get(i)).text();
|
||||
s = (terms.get(i)).text();
|
||||
} else {
|
||||
s = s + "|" + ((Term)terms.get(i)).text();
|
||||
s = s + "|" + (terms.get(i)).text();
|
||||
}
|
||||
pieces[pos] = s;
|
||||
}
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.lucene.search;
|
|||
import java.io.IOException;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
@ -126,7 +126,7 @@ public abstract class Query implements java.io.Serializable, Cloneable {
|
|||
* the other queries.
|
||||
*/
|
||||
public Query combine(Query[] queries) {
|
||||
HashSet uniques = new HashSet();
|
||||
HashSet<Query> uniques = new HashSet<Query>();
|
||||
for (int i = 0; i < queries.length; i++) {
|
||||
Query query = queries[i];
|
||||
BooleanClause[] clauses = null;
|
||||
|
@ -152,10 +152,9 @@ public abstract class Query implements java.io.Serializable, Cloneable {
|
|||
if(uniques.size() == 1){
|
||||
return (Query)uniques.iterator().next();
|
||||
}
|
||||
Iterator it = uniques.iterator();
|
||||
BooleanQuery result = new BooleanQuery(true);
|
||||
while (it.hasNext())
|
||||
result.add((Query) it.next(), BooleanClause.Occur.SHOULD);
|
||||
for (final Query query : uniques)
|
||||
result.add(query, BooleanClause.Occur.SHOULD);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -179,20 +178,18 @@ public abstract class Query implements java.io.Serializable, Cloneable {
|
|||
*<p>A utility for use by {@link #combine(Query[])} implementations.
|
||||
*/
|
||||
public static Query mergeBooleanQueries(BooleanQuery[] queries) {
|
||||
HashSet allClauses = new HashSet();
|
||||
for (int i = 0; i < queries.length; i++) {
|
||||
BooleanClause[] clauses = queries[i].getClauses();
|
||||
for (int j = 0; j < clauses.length; j++) {
|
||||
allClauses.add(clauses[j]);
|
||||
HashSet<BooleanClause> allClauses = new HashSet<BooleanClause>();
|
||||
for (BooleanQuery booleanQuery : queries) {
|
||||
for (BooleanClause clause : booleanQuery) {
|
||||
allClauses.add(clause);
|
||||
}
|
||||
}
|
||||
|
||||
boolean coordDisabled =
|
||||
queries.length==0? false : queries[0].isCoordDisabled();
|
||||
BooleanQuery result = new BooleanQuery(coordDisabled);
|
||||
Iterator i = allClauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
result.add((BooleanClause)i.next());
|
||||
for(BooleanClause clause2 : allClauses) {
|
||||
result.add(clause2);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.StringReader;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -56,7 +56,7 @@ public class QueryTermVector implements TermFreqVector {
|
|||
TokenStream stream = analyzer.tokenStream("", new StringReader(queryString));
|
||||
if (stream != null)
|
||||
{
|
||||
List terms = new ArrayList();
|
||||
List<String> terms = new ArrayList<String>();
|
||||
try {
|
||||
boolean hasMoreTokens = false;
|
||||
|
||||
|
@ -78,30 +78,29 @@ public class QueryTermVector implements TermFreqVector {
|
|||
private void processTerms(String[] queryTerms) {
|
||||
if (queryTerms != null) {
|
||||
Arrays.sort(queryTerms);
|
||||
Map tmpSet = new HashMap(queryTerms.length);
|
||||
Map<String,Integer> tmpSet = new HashMap<String,Integer>(queryTerms.length);
|
||||
//filter out duplicates
|
||||
List tmpList = new ArrayList(queryTerms.length);
|
||||
List tmpFreqs = new ArrayList(queryTerms.length);
|
||||
List<String> tmpList = new ArrayList<String>(queryTerms.length);
|
||||
List<Integer> tmpFreqs = new ArrayList<Integer>(queryTerms.length);
|
||||
int j = 0;
|
||||
for (int i = 0; i < queryTerms.length; i++) {
|
||||
String term = queryTerms[i];
|
||||
Integer position = (Integer)tmpSet.get(term);
|
||||
Integer position = tmpSet.get(term);
|
||||
if (position == null) {
|
||||
tmpSet.put(term, Integer.valueOf(j++));
|
||||
tmpList.add(term);
|
||||
tmpFreqs.add(Integer.valueOf(1));
|
||||
}
|
||||
else {
|
||||
Integer integer = (Integer)tmpFreqs.get(position.intValue());
|
||||
Integer integer = tmpFreqs.get(position.intValue());
|
||||
tmpFreqs.set(position.intValue(), Integer.valueOf(integer.intValue() + 1));
|
||||
}
|
||||
}
|
||||
terms = (String[])tmpList.toArray(terms);
|
||||
terms = tmpList.toArray(terms);
|
||||
//termFreqs = (int[])tmpFreqs.toArray(termFreqs);
|
||||
termFreqs = new int[tmpFreqs.size()];
|
||||
int i = 0;
|
||||
for (Iterator iter = tmpFreqs.iterator(); iter.hasNext();) {
|
||||
Integer integer = (Integer) iter.next();
|
||||
for (final Integer integer : tmpFreqs) {
|
||||
termFreqs[i++] = integer.intValue();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.io.IOException;
|
|||
import java.io.Serializable;
|
||||
import java.util.Collection;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
|
||||
|
||||
/**
|
||||
* Expert: Scoring API.
|
||||
|
@ -787,11 +787,11 @@ public abstract class Similarity implements Serializable {
|
|||
* @return idf score factor
|
||||
* @deprecated see {@link #idfExplain(Collection, Searcher)}
|
||||
*/
|
||||
public float idf(Collection terms, Searcher searcher) throws IOException {
|
||||
public float idf(Collection<Term> terms, Searcher searcher) throws IOException {
|
||||
float idf = 0.0f;
|
||||
Iterator i = terms.iterator();
|
||||
while (i.hasNext()) {
|
||||
idf += idf((Term)i.next(), searcher);
|
||||
|
||||
for(final Term term: terms) {
|
||||
idf += idf(term, searcher);
|
||||
}
|
||||
return idf;
|
||||
}
|
||||
|
@ -810,7 +810,7 @@ public abstract class Similarity implements Serializable {
|
|||
* for each term.
|
||||
* @throws IOException
|
||||
*/
|
||||
public IDFExplanation idfExplain(Collection terms, Searcher searcher) throws IOException {
|
||||
public IDFExplanation idfExplain(Collection<Term> terms, Searcher searcher) throws IOException {
|
||||
if(supportedMethods.overridesCollectionIDF) {
|
||||
final float idf = idf(terms, searcher);
|
||||
return new IDFExplanation() {
|
||||
|
@ -827,9 +827,7 @@ public abstract class Similarity implements Serializable {
|
|||
final int max = searcher.maxDoc();
|
||||
float idf = 0.0f;
|
||||
final StringBuilder exp = new StringBuilder();
|
||||
Iterator i = terms.iterator();
|
||||
while (i.hasNext()) {
|
||||
Term term = (Term)i.next();
|
||||
for (final Term term : terms ) {
|
||||
final int df = searcher.docFreq(term);
|
||||
idf += idf(df, max);
|
||||
exp.append(" ");
|
||||
|
@ -955,7 +953,7 @@ public abstract class Similarity implements Serializable {
|
|||
}
|
||||
|
||||
/** @deprecated Remove this when old API is removed! */
|
||||
private static final IdentityHashMap/*<Class<? extends Similarity>,MethodSupport>*/ knownMethodSupport = new IdentityHashMap();
|
||||
private static final IdentityHashMap<Class<? extends Similarity>,MethodSupport> knownMethodSupport = new IdentityHashMap();
|
||||
|
||||
/** @deprecated Remove this when old API is removed! */
|
||||
private static MethodSupport getSupportedMethods(Class clazz) {
|
||||
|
|
|
@ -146,14 +146,14 @@ final class SloppyPhraseScorer extends PhraseScorer {
|
|||
if (!checkedRepeats) {
|
||||
checkedRepeats = true;
|
||||
// check for repeats
|
||||
HashMap m = null;
|
||||
HashMap<PhrasePositions, Object> m = null;
|
||||
for (PhrasePositions pp = first; pp != null; pp = pp.next) {
|
||||
int tpPos = pp.position + pp.offset;
|
||||
for (PhrasePositions pp2 = pp.next; pp2 != null; pp2 = pp2.next) {
|
||||
int tpPos2 = pp2.position + pp2.offset;
|
||||
if (tpPos2 == tpPos) {
|
||||
if (m == null)
|
||||
m = new HashMap();
|
||||
m = new HashMap<PhrasePositions, Object>();
|
||||
pp.repeats = true;
|
||||
pp2.repeats = true;
|
||||
m.put(pp,null);
|
||||
|
@ -162,7 +162,7 @@ final class SloppyPhraseScorer extends PhraseScorer {
|
|||
}
|
||||
}
|
||||
if (m!=null)
|
||||
repeats = (PhrasePositions[]) m.keySet().toArray(new PhrasePositions[0]);
|
||||
repeats = m.keySet().toArray(new PhrasePositions[0]);
|
||||
}
|
||||
|
||||
// with repeats must advance some repeating pp's so they all start with differing tp's
|
||||
|
|
|
@ -16,7 +16,7 @@ package org.apache.lucene.search;
|
|||
*/
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
||||
|
@ -29,14 +29,14 @@ import java.util.List;
|
|||
**/
|
||||
public class SpanFilterResult {
|
||||
private DocIdSet docIdSet;
|
||||
private List positions;//Spans spans;
|
||||
private List<PositionInfo> positions;//Spans spans;
|
||||
|
||||
/**
|
||||
*
|
||||
* @param docIdSet The DocIdSet for the Filter
|
||||
* @param positions A List of {@link org.apache.lucene.search.SpanFilterResult.PositionInfo} objects
|
||||
*/
|
||||
public SpanFilterResult(DocIdSet docIdSet, List positions) {
|
||||
public SpanFilterResult(DocIdSet docIdSet, List<PositionInfo> positions) {
|
||||
this.docIdSet = docIdSet;
|
||||
this.positions = positions;
|
||||
}
|
||||
|
@ -46,7 +46,7 @@ public class SpanFilterResult {
|
|||
* Entries are increasing by document order
|
||||
* @return A List of PositionInfo objects
|
||||
*/
|
||||
public List getPositions() {
|
||||
public List<PositionInfo> getPositions() {
|
||||
return positions;
|
||||
}
|
||||
|
||||
|
@ -57,12 +57,12 @@ public class SpanFilterResult {
|
|||
|
||||
public static class PositionInfo {
|
||||
private int doc;
|
||||
private List positions;
|
||||
private List<StartEnd> positions;
|
||||
|
||||
|
||||
public PositionInfo(int doc) {
|
||||
this.doc = doc;
|
||||
positions = new ArrayList();
|
||||
positions = new ArrayList<StartEnd>();
|
||||
}
|
||||
|
||||
public void addPosition(int start, int end)
|
||||
|
@ -76,9 +76,9 @@ public class SpanFilterResult {
|
|||
|
||||
/**
|
||||
*
|
||||
* @return A List of {@link org.apache.lucene.search.SpanFilterResult.StartEnd} objects
|
||||
* @return Positions
|
||||
*/
|
||||
public List getPositions() {
|
||||
public List<StartEnd> getPositions() {
|
||||
return positions;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,7 +63,7 @@ public class SpanQueryFilter extends SpanFilter {
|
|||
|
||||
final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
|
||||
Spans spans = query.getSpans(reader);
|
||||
List tmp = new ArrayList(20);
|
||||
List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
|
||||
int currentDoc = -1;
|
||||
SpanFilterResult.PositionInfo currentInfo = null;
|
||||
while (spans.next())
|
||||
|
|
|
@ -77,8 +77,7 @@ public class PayloadNearQuery extends SpanNearQuery {
|
|||
SpanQuery[] newClauses = new SpanQuery[sz];
|
||||
|
||||
for (int i = 0; i < sz; i++) {
|
||||
SpanQuery clause = (SpanQuery) clauses.get(i);
|
||||
newClauses[i] = (SpanQuery) clause.clone();
|
||||
newClauses[i] = (SpanQuery) clauses.get(i).clone();
|
||||
}
|
||||
PayloadNearQuery boostingNearQuery = new PayloadNearQuery(newClauses, slop,
|
||||
inOrder);
|
||||
|
@ -90,9 +89,9 @@ public class PayloadNearQuery extends SpanNearQuery {
|
|||
public String toString(String field) {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
buffer.append("payloadNear([");
|
||||
Iterator i = clauses.iterator();
|
||||
Iterator<SpanQuery> i = clauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
SpanQuery clause = (SpanQuery) i.next();
|
||||
SpanQuery clause = i.next();
|
||||
buffer.append(clause.toString(field));
|
||||
if (i.hasNext()) {
|
||||
buffer.append(", ");
|
||||
|
@ -194,9 +193,8 @@ public class PayloadNearQuery extends SpanNearQuery {
|
|||
*
|
||||
* @see Spans
|
||||
*/
|
||||
protected void processPayloads(Collection payLoads, int start, int end) {
|
||||
for (Iterator iterator = payLoads.iterator(); iterator.hasNext();) {
|
||||
byte[] thePayload = (byte[]) iterator.next();
|
||||
protected void processPayloads(Collection<byte[]> payLoads, int start, int end) {
|
||||
for (final byte[] thePayload : payLoads) {
|
||||
payloadScore = function.currentScore(doc, fieldName, start, end,
|
||||
payloadsSeen, payloadScore, similarity.scorePayload(doc, fieldName,
|
||||
spans.start(), spans.end(), thePayload, 0, thePayload.length));
|
||||
|
|
|
@ -69,13 +69,13 @@ public class PayloadSpanUtil {
|
|||
* @return payloads Collection
|
||||
* @throws IOException
|
||||
*/
|
||||
public Collection getPayloadsForQuery(Query query) throws IOException {
|
||||
Collection payloads = new ArrayList();
|
||||
public Collection<byte[]> getPayloadsForQuery(Query query) throws IOException {
|
||||
Collection<byte[]> payloads = new ArrayList<byte[]>();
|
||||
queryToSpanQuery(query, payloads);
|
||||
return payloads;
|
||||
}
|
||||
|
||||
private void queryToSpanQuery(Query query, Collection payloads)
|
||||
private void queryToSpanQuery(Query query, Collection<byte[]> payloads)
|
||||
throws IOException {
|
||||
if (query instanceof BooleanQuery) {
|
||||
BooleanClause[] queryClauses = ((BooleanQuery) query).getClauses();
|
||||
|
@ -113,14 +113,14 @@ public class PayloadSpanUtil {
|
|||
queryToSpanQuery(((FilteredQuery) query).getQuery(), payloads);
|
||||
} else if (query instanceof DisjunctionMaxQuery) {
|
||||
|
||||
for (Iterator iterator = ((DisjunctionMaxQuery) query).iterator(); iterator
|
||||
for (Iterator<Query> iterator = ((DisjunctionMaxQuery) query).iterator(); iterator
|
||||
.hasNext();) {
|
||||
queryToSpanQuery((Query) iterator.next(), payloads);
|
||||
queryToSpanQuery(iterator.next(), payloads);
|
||||
}
|
||||
|
||||
} else if (query instanceof MultiPhraseQuery) {
|
||||
final MultiPhraseQuery mpq = (MultiPhraseQuery) query;
|
||||
final List termArrays = mpq.getTermArrays();
|
||||
final List<Term[]> termArrays = mpq.getTermArrays();
|
||||
final int[] positions = mpq.getPositions();
|
||||
if (positions.length > 0) {
|
||||
|
||||
|
@ -131,19 +131,19 @@ public class PayloadSpanUtil {
|
|||
}
|
||||
}
|
||||
|
||||
final List[] disjunctLists = new List[maxPosition + 1];
|
||||
final List<Query>[] disjunctLists = new List[maxPosition + 1];
|
||||
int distinctPositions = 0;
|
||||
|
||||
for (int i = 0; i < termArrays.size(); ++i) {
|
||||
final Term[] termArray = (Term[]) termArrays.get(i);
|
||||
List disjuncts = disjunctLists[positions[i]];
|
||||
final Term[] termArray = termArrays.get(i);
|
||||
List<Query> disjuncts = disjunctLists[positions[i]];
|
||||
if (disjuncts == null) {
|
||||
disjuncts = (disjunctLists[positions[i]] = new ArrayList(
|
||||
disjuncts = (disjunctLists[positions[i]] = new ArrayList<Query>(
|
||||
termArray.length));
|
||||
++distinctPositions;
|
||||
}
|
||||
for (int j = 0; j < termArray.length; ++j) {
|
||||
disjuncts.add(new SpanTermQuery(termArray[j]));
|
||||
for (final Term term : termArray) {
|
||||
disjuncts.add(new SpanTermQuery(term));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -151,9 +151,9 @@ public class PayloadSpanUtil {
|
|||
int position = 0;
|
||||
final SpanQuery[] clauses = new SpanQuery[distinctPositions];
|
||||
for (int i = 0; i < disjunctLists.length; ++i) {
|
||||
List disjuncts = disjunctLists[i];
|
||||
List<Query> disjuncts = disjunctLists[i];
|
||||
if (disjuncts != null) {
|
||||
clauses[position++] = new SpanOrQuery((SpanQuery[]) disjuncts
|
||||
clauses[position++] = new SpanOrQuery(disjuncts
|
||||
.toArray(new SpanQuery[disjuncts.size()]));
|
||||
} else {
|
||||
++positionGaps;
|
||||
|
@ -171,16 +171,14 @@ public class PayloadSpanUtil {
|
|||
}
|
||||
}
|
||||
|
||||
private void getPayloads(Collection payloads, SpanQuery query)
|
||||
private void getPayloads(Collection<byte []> payloads, SpanQuery query)
|
||||
throws IOException {
|
||||
Spans spans = query.getSpans(reader);
|
||||
|
||||
while (spans.next() == true) {
|
||||
if (spans.isPayloadAvailable()) {
|
||||
Collection payload = spans.getPayload();
|
||||
Iterator it = payload.iterator();
|
||||
while (it.hasNext()) {
|
||||
byte[] bytes = (byte[]) it.next();
|
||||
Collection<byte[]> payload = spans.getPayload();
|
||||
for (byte [] bytes : payload) {
|
||||
payloads.add(bytes);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.lucene.search.spans;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
|
|
@ -65,12 +65,12 @@ public class NearSpansOrdered extends Spans {
|
|||
private int matchDoc = -1;
|
||||
private int matchStart = -1;
|
||||
private int matchEnd = -1;
|
||||
private List/*<byte[]>*/ matchPayload;
|
||||
private List<byte[]> matchPayload;
|
||||
|
||||
private final Spans[] subSpansByDoc;
|
||||
private final Comparator spanDocComparator = new Comparator() {
|
||||
public int compare(Object o1, Object o2) {
|
||||
return ((Spans)o1).doc() - ((Spans)o2).doc();
|
||||
private final Comparator<Spans> spanDocComparator = new Comparator<Spans>() {
|
||||
public int compare(Spans o1, Spans o2) {
|
||||
return o1.doc() - o2.doc();
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -91,7 +91,7 @@ public class NearSpansOrdered extends Spans {
|
|||
allowedSlop = spanNearQuery.getSlop();
|
||||
SpanQuery[] clauses = spanNearQuery.getClauses();
|
||||
subSpans = new Spans[clauses.length];
|
||||
matchPayload = new LinkedList();
|
||||
matchPayload = new LinkedList<byte[]>();
|
||||
subSpansByDoc = new Spans[clauses.length];
|
||||
for (int i = 0; i < clauses.length; i++) {
|
||||
subSpans[i] = clauses[i].getSpans(reader);
|
||||
|
@ -115,7 +115,7 @@ public class NearSpansOrdered extends Spans {
|
|||
|
||||
// TODO: Remove warning after API has been finalized
|
||||
// TODO: Would be nice to be able to lazy load payloads
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
return matchPayload;
|
||||
}
|
||||
|
||||
|
@ -256,12 +256,12 @@ public class NearSpansOrdered extends Spans {
|
|||
private boolean shrinkToAfterShortestMatch() throws IOException {
|
||||
matchStart = subSpans[subSpans.length - 1].start();
|
||||
matchEnd = subSpans[subSpans.length - 1].end();
|
||||
Set possibleMatchPayloads = new HashSet();
|
||||
Set<byte[]> possibleMatchPayloads = new HashSet<byte[]>();
|
||||
if (subSpans[subSpans.length - 1].isPayloadAvailable()) {
|
||||
possibleMatchPayloads.addAll(subSpans[subSpans.length - 1].getPayload());
|
||||
}
|
||||
|
||||
Collection possiblePayload = null;
|
||||
Collection<byte[]> possiblePayload = null;
|
||||
|
||||
int matchSlop = 0;
|
||||
int lastStart = matchStart;
|
||||
|
@ -269,8 +269,8 @@ public class NearSpansOrdered extends Spans {
|
|||
for (int i = subSpans.length - 2; i >= 0; i--) {
|
||||
Spans prevSpans = subSpans[i];
|
||||
if (collectPayloads && prevSpans.isPayloadAvailable()) {
|
||||
Collection payload = prevSpans.getPayload();
|
||||
possiblePayload = new ArrayList(payload.size());
|
||||
Collection<byte[]> payload = prevSpans.getPayload();
|
||||
possiblePayload = new ArrayList<byte[]>(payload.size());
|
||||
possiblePayload.addAll(payload);
|
||||
}
|
||||
|
||||
|
@ -293,8 +293,8 @@ public class NearSpansOrdered extends Spans {
|
|||
prevStart = ppStart;
|
||||
prevEnd = ppEnd;
|
||||
if (collectPayloads && prevSpans.isPayloadAvailable()) {
|
||||
Collection payload = prevSpans.getPayload();
|
||||
possiblePayload = new ArrayList(payload.size());
|
||||
Collection<byte[]> payload = prevSpans.getPayload();
|
||||
possiblePayload = new ArrayList<byte[]>(payload.size());
|
||||
possiblePayload.addAll(payload);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ import java.util.HashSet;
|
|||
public class NearSpansUnordered extends Spans {
|
||||
private SpanNearQuery query;
|
||||
|
||||
private List ordered = new ArrayList(); // spans in query order
|
||||
private List<SpansCell> ordered = new ArrayList<SpansCell>(); // spans in query order
|
||||
private Spans[] subSpans;
|
||||
private int slop; // from query
|
||||
|
||||
|
@ -107,8 +107,8 @@ public class NearSpansUnordered extends Spans {
|
|||
public int start() { return spans.start(); }
|
||||
public int end() { return spans.end(); }
|
||||
// TODO: Remove warning after API has been finalized
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
return new ArrayList(spans.getPayload());
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
return new ArrayList<byte[]>(spans.getPayload());
|
||||
}
|
||||
|
||||
// TODO: Remove warning after API has been finalized
|
||||
|
@ -223,8 +223,8 @@ public class NearSpansUnordered extends Spans {
|
|||
* @return Collection of <code>byte[]</code> payloads
|
||||
* @throws IOException
|
||||
*/
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
Set/*<byte[]*/ matchPayload = new HashSet();
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
Set<byte[]> matchPayload = new HashSet<byte[]>();
|
||||
for (SpansCell cell = first; cell != null; cell = cell.next) {
|
||||
if (cell.isPayloadAvailable()) {
|
||||
matchPayload.addAll(cell.getPayload());
|
||||
|
@ -253,7 +253,7 @@ public class NearSpansUnordered extends Spans {
|
|||
|
||||
private void initList(boolean next) throws IOException {
|
||||
for (int i = 0; more && i < ordered.size(); i++) {
|
||||
SpansCell cell = (SpansCell)ordered.get(i);
|
||||
SpansCell cell = ordered.get(i);
|
||||
if (next)
|
||||
more = cell.next(); // move to first entry
|
||||
if (more) {
|
||||
|
|
|
@ -94,10 +94,10 @@ public class SpanFirstQuery extends SpanQuery implements Cloneable {
|
|||
public int end() { return spans.end(); }
|
||||
|
||||
// TODO: Remove warning after API has been finalized
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
ArrayList result = null;
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
ArrayList<byte[]> result = null;
|
||||
if (spans.isPayloadAvailable()) {
|
||||
result = new ArrayList(spans.getPayload());
|
||||
result = new ArrayList<byte[]>(spans.getPayload());
|
||||
}
|
||||
return result;//TODO: any way to avoid the new construction?
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.lucene.search.spans;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
|
@ -35,7 +35,7 @@ import org.apache.lucene.util.ToStringUtils;
|
|||
* maximum number of intervening unmatched positions, as well as whether
|
||||
* matches are required to be in-order. */
|
||||
public class SpanNearQuery extends SpanQuery implements Cloneable {
|
||||
protected List clauses;
|
||||
protected List<SpanQuery> clauses;
|
||||
protected int slop;
|
||||
protected boolean inOrder;
|
||||
|
||||
|
@ -53,7 +53,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
public SpanNearQuery(SpanQuery[] clauses, int slop, boolean inOrder, boolean collectPayloads) {
|
||||
|
||||
// copy clauses array into an ArrayList
|
||||
this.clauses = new ArrayList(clauses.length);
|
||||
this.clauses = new ArrayList<SpanQuery>(clauses.length);
|
||||
for (int i = 0; i < clauses.length; i++) {
|
||||
SpanQuery clause = clauses[i];
|
||||
if (i == 0) { // check field
|
||||
|
@ -70,7 +70,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
|
||||
/** Return the clauses whose spans are matched. */
|
||||
public SpanQuery[] getClauses() {
|
||||
return (SpanQuery[])clauses.toArray(new SpanQuery[clauses.size()]);
|
||||
return clauses.toArray(new SpanQuery[clauses.size()]);
|
||||
}
|
||||
|
||||
/** Return the maximum number of intervening unmatched positions permitted.*/
|
||||
|
@ -82,9 +82,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
public String getField() { return field; }
|
||||
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
Iterator i = clauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
SpanQuery clause = (SpanQuery)i.next();
|
||||
for (final SpanQuery clause : clauses) {
|
||||
clause.extractTerms(terms);
|
||||
}
|
||||
}
|
||||
|
@ -93,9 +91,9 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
public String toString(String field) {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
buffer.append("spanNear([");
|
||||
Iterator i = clauses.iterator();
|
||||
Iterator<SpanQuery> i = clauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
SpanQuery clause = (SpanQuery)i.next();
|
||||
SpanQuery clause = i.next();
|
||||
buffer.append(clause.toString(field));
|
||||
if (i.hasNext()) {
|
||||
buffer.append(", ");
|
||||
|
@ -115,7 +113,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
return new SpanOrQuery(getClauses()).getSpans(reader);
|
||||
|
||||
if (clauses.size() == 1) // optimize 1-clause case
|
||||
return ((SpanQuery)clauses.get(0)).getSpans(reader);
|
||||
return clauses.get(0).getSpans(reader);
|
||||
|
||||
return inOrder
|
||||
? (Spans) new NearSpansOrdered(this, reader, collectPayloads)
|
||||
|
@ -125,7 +123,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
SpanNearQuery clone = null;
|
||||
for (int i = 0 ; i < clauses.size(); i++) {
|
||||
SpanQuery c = (SpanQuery)clauses.get(i);
|
||||
SpanQuery c = clauses.get(i);
|
||||
SpanQuery query = (SpanQuery) c.rewrite(reader);
|
||||
if (query != c) { // clause rewrote: must clone
|
||||
if (clone == null)
|
||||
|
@ -145,8 +143,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
|
|||
SpanQuery[] newClauses = new SpanQuery[sz];
|
||||
|
||||
for (int i = 0; i < sz; i++) {
|
||||
SpanQuery clause = (SpanQuery) clauses.get(i);
|
||||
newClauses[i] = (SpanQuery) clause.clone();
|
||||
newClauses[i] = (SpanQuery) clauses.get(i).clone();
|
||||
}
|
||||
SpanNearQuery spanNearQuery = new SpanNearQuery(newClauses, slop, inOrder);
|
||||
spanNearQuery.setBoost(getBoost());
|
||||
|
|
|
@ -132,10 +132,10 @@ public class SpanNotQuery extends SpanQuery implements Cloneable {
|
|||
public int end() { return includeSpans.end(); }
|
||||
|
||||
// TODO: Remove warning after API has been finalized
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
ArrayList result = null;
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
ArrayList<byte[]> result = null;
|
||||
if (includeSpans.isPayloadAvailable()) {
|
||||
result = new ArrayList(includeSpans.getPayload());
|
||||
result = new ArrayList<byte[]>(includeSpans.getPayload());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -33,14 +33,14 @@ import org.apache.lucene.search.Query;
|
|||
|
||||
/** Matches the union of its clauses.*/
|
||||
public class SpanOrQuery extends SpanQuery implements Cloneable {
|
||||
private List clauses;
|
||||
private List<SpanQuery> clauses;
|
||||
private String field;
|
||||
|
||||
/** Construct a SpanOrQuery merging the provided clauses. */
|
||||
public SpanOrQuery(SpanQuery[] clauses) {
|
||||
|
||||
// copy clauses array into an ArrayList
|
||||
this.clauses = new ArrayList(clauses.length);
|
||||
this.clauses = new ArrayList<SpanQuery>(clauses.length);
|
||||
for (int i = 0; i < clauses.length; i++) {
|
||||
SpanQuery clause = clauses[i];
|
||||
if (i == 0) { // check field
|
||||
|
@ -60,9 +60,7 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
public String getField() { return field; }
|
||||
|
||||
public void extractTerms(Set<Term> terms) {
|
||||
Iterator i = clauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
SpanQuery clause = (SpanQuery)i.next();
|
||||
for(final SpanQuery clause: clauses) {
|
||||
clause.extractTerms(terms);
|
||||
}
|
||||
}
|
||||
|
@ -72,8 +70,7 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
SpanQuery[] newClauses = new SpanQuery[sz];
|
||||
|
||||
for (int i = 0; i < sz; i++) {
|
||||
SpanQuery clause = (SpanQuery) clauses.get(i);
|
||||
newClauses[i] = (SpanQuery) clause.clone();
|
||||
newClauses[i] = (SpanQuery) clauses.get(i).clone();
|
||||
}
|
||||
SpanOrQuery soq = new SpanOrQuery(newClauses);
|
||||
soq.setBoost(getBoost());
|
||||
|
@ -83,7 +80,7 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
public Query rewrite(IndexReader reader) throws IOException {
|
||||
SpanOrQuery clone = null;
|
||||
for (int i = 0 ; i < clauses.size(); i++) {
|
||||
SpanQuery c = (SpanQuery)clauses.get(i);
|
||||
SpanQuery c = clauses.get(i);
|
||||
SpanQuery query = (SpanQuery) c.rewrite(reader);
|
||||
if (query != c) { // clause rewrote: must clone
|
||||
if (clone == null)
|
||||
|
@ -101,9 +98,9 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
public String toString(String field) {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
buffer.append("spanOr([");
|
||||
Iterator i = clauses.iterator();
|
||||
Iterator<SpanQuery> i = clauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
SpanQuery clause = (SpanQuery)i.next();
|
||||
SpanQuery clause = i.next();
|
||||
buffer.append(clause.toString(field));
|
||||
if (i.hasNext()) {
|
||||
buffer.append(", ");
|
||||
|
@ -134,14 +131,12 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
}
|
||||
|
||||
|
||||
private class SpanQueue extends PriorityQueue {
|
||||
private class SpanQueue extends PriorityQueue<Spans> {
|
||||
public SpanQueue(int size) {
|
||||
initialize(size);
|
||||
}
|
||||
|
||||
protected final boolean lessThan(Object o1, Object o2) {
|
||||
Spans spans1 = (Spans)o1;
|
||||
Spans spans2 = (Spans)o2;
|
||||
protected final boolean lessThan(Spans spans1, Spans spans2) {
|
||||
if (spans1.doc() == spans2.doc()) {
|
||||
if (spans1.start() == spans2.start()) {
|
||||
return spans1.end() < spans2.end();
|
||||
|
@ -163,9 +158,9 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
|
||||
private boolean initSpanQueue(int target) throws IOException {
|
||||
queue = new SpanQueue(clauses.size());
|
||||
Iterator i = clauses.iterator();
|
||||
Iterator<SpanQuery> i = clauses.iterator();
|
||||
while (i.hasNext()) {
|
||||
Spans spans = ((SpanQuery)i.next()).getSpans(reader);
|
||||
Spans spans = i.next().getSpans(reader);
|
||||
if ( ((target == -1) && spans.next())
|
||||
|| ((target != -1) && spans.skipTo(target))) {
|
||||
queue.add(spans);
|
||||
|
@ -219,11 +214,11 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
|
|||
public int start() { return top().start(); }
|
||||
public int end() { return top().end(); }
|
||||
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
ArrayList result = null;
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
ArrayList<byte[]> result = null;
|
||||
Spans theTop = top();
|
||||
if (theTop != null && theTop.isPayloadAvailable()) {
|
||||
result = new ArrayList(theTop.getPayload());
|
||||
result = new ArrayList<byte[]>(theTop.getPayload());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@ package org.apache.lucene.search.spans;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
|
|
@ -22,8 +22,6 @@ import org.apache.lucene.index.Term;
|
|||
import org.apache.lucene.util.ToStringUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Set;
|
||||
|
||||
/** Matches spans containing a term. */
|
||||
|
|
|
@ -18,6 +18,7 @@ package org.apache.lucene.search.spans;
|
|||
*/
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.search.Explanation.IDFExplanation;
|
||||
|
||||
|
@ -35,7 +36,7 @@ public class SpanWeight extends Weight {
|
|||
protected float queryNorm;
|
||||
protected float queryWeight;
|
||||
|
||||
protected Set terms;
|
||||
protected Set<Term> terms;
|
||||
protected SpanQuery query;
|
||||
private IDFExplanation idfExp;
|
||||
|
||||
|
@ -43,8 +44,10 @@ public class SpanWeight extends Weight {
|
|||
throws IOException {
|
||||
this.similarity = query.getSimilarity(searcher);
|
||||
this.query = query;
|
||||
terms=new HashSet();
|
||||
|
||||
terms=new HashSet<Term>();
|
||||
query.extractTerms(terms);
|
||||
|
||||
idfExp = similarity.idfExplain(terms, searcher);
|
||||
idf = idfExp.getIdf();
|
||||
}
|
||||
|
|
|
@ -75,7 +75,7 @@ public abstract class Spans {
|
|||
* @throws java.io.IOException
|
||||
*/
|
||||
// TODO: Remove warning after API has been finalized
|
||||
public abstract Collection/*<byte[]>*/ getPayload() throws IOException;
|
||||
public abstract Collection<byte[]> getPayload() throws IOException;
|
||||
|
||||
/**
|
||||
* Checks if a payload can be loaded at this position.
|
||||
|
|
|
@ -87,7 +87,7 @@ public class TermSpans extends Spans {
|
|||
}
|
||||
|
||||
// TODO: Remove warning after API has been finalized
|
||||
public Collection/*<byte[]>*/ getPayload() throws IOException {
|
||||
public Collection<byte[]> getPayload() throws IOException {
|
||||
byte [] bytes = new byte[positions.getPayloadLength()];
|
||||
bytes = positions.getPayload(bytes, 0);
|
||||
return Collections.singletonList(bytes);
|
||||
|
|
|
@ -19,14 +19,10 @@ package org.apache.lucene.store;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.Constants;
|
||||
|
||||
|
|
|
@ -18,8 +18,7 @@ package org.apache.lucene.store;
|
|||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -39,10 +38,10 @@ import java.util.Set;
|
|||
public class FileSwitchDirectory extends Directory {
|
||||
private final Directory secondaryDir;
|
||||
private final Directory primaryDir;
|
||||
private final Set primaryExtensions;
|
||||
private final Set<String> primaryExtensions;
|
||||
private boolean doClose;
|
||||
|
||||
public FileSwitchDirectory(Set primaryExtensions, Directory primaryDir, Directory secondaryDir, boolean doClose) {
|
||||
public FileSwitchDirectory(Set<String> primaryExtensions, Directory primaryDir, Directory secondaryDir, boolean doClose) {
|
||||
this.primaryExtensions = primaryExtensions;
|
||||
this.primaryDir = primaryDir;
|
||||
this.secondaryDir = secondaryDir;
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.lucene.store;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Iterator;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
|
||||
/** Abstract base class for output to a file in a Directory. A random-access
|
||||
|
@ -210,17 +209,14 @@ public abstract class IndexOutput {
|
|||
*/
|
||||
public void setLength(long length) throws IOException {};
|
||||
|
||||
// map must be Map<String, String>
|
||||
public void writeStringStringMap(Map map) throws IOException {
|
||||
public void writeStringStringMap(Map<String,String> map) throws IOException {
|
||||
if (map == null) {
|
||||
writeInt(0);
|
||||
} else {
|
||||
writeInt(map.size());
|
||||
final Iterator it = map.entrySet().iterator();
|
||||
while(it.hasNext()) {
|
||||
Map.Entry entry = (Map.Entry) it.next();
|
||||
writeString((String) entry.getKey());
|
||||
writeString((String) entry.getValue());
|
||||
for(final Map.Entry<String, String> entry: map.entrySet()) {
|
||||
writeString(entry.getKey());
|
||||
writeString(entry.getValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -148,7 +148,7 @@ public class MMapDirectory extends FSDirectory {
|
|||
final void cleanMapping(final ByteBuffer buffer) throws IOException {
|
||||
if (useUnmapHack) {
|
||||
try {
|
||||
AccessController.doPrivileged(new PrivilegedExceptionAction() {
|
||||
AccessController.doPrivileged(new PrivilegedExceptionAction<Object>() {
|
||||
public Object run() throws Exception {
|
||||
final Method getCleanerMethod = buffer.getClass()
|
||||
.getMethod("cleaner", NO_PARAM_TYPES);
|
||||
|
|
|
@ -164,7 +164,7 @@ class NativeFSLock extends Lock {
|
|||
* one JVM (each with their own NativeFSLockFactory
|
||||
* instance) have set the same lock dir and lock prefix.
|
||||
*/
|
||||
private static HashSet LOCK_HELD = new HashSet();
|
||||
private static HashSet<String> LOCK_HELD = new HashSet<String>();
|
||||
|
||||
public NativeFSLock(File lockDir, String lockFileName) {
|
||||
this.lockDir = lockDir;
|
||||
|
|
|
@ -19,10 +19,8 @@ package org.apache.lucene.store;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.File;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -34,7 +32,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
|
||||
private static final long serialVersionUID = 1l;
|
||||
|
||||
HashMap fileMap = new HashMap();
|
||||
HashMap<String,RAMFile> fileMap = new HashMap<String,RAMFile>();
|
||||
long sizeInBytes = 0;
|
||||
|
||||
// *****
|
||||
|
@ -73,12 +71,11 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
|
||||
public synchronized final String[] listAll() {
|
||||
ensureOpen();
|
||||
Set fileNames = fileMap.keySet();
|
||||
Set<String> fileNames = fileMap.keySet();
|
||||
String[] result = new String[fileNames.size()];
|
||||
int i = 0;
|
||||
Iterator it = fileNames.iterator();
|
||||
while (it.hasNext())
|
||||
result[i++] = (String)it.next();
|
||||
for(final String fileName: fileNames)
|
||||
result[i++] = fileName;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -87,7 +84,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
ensureOpen();
|
||||
RAMFile file;
|
||||
synchronized (this) {
|
||||
file = (RAMFile)fileMap.get(name);
|
||||
file = fileMap.get(name);
|
||||
}
|
||||
return file != null;
|
||||
}
|
||||
|
@ -99,7 +96,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
ensureOpen();
|
||||
RAMFile file;
|
||||
synchronized (this) {
|
||||
file = (RAMFile)fileMap.get(name);
|
||||
file = fileMap.get(name);
|
||||
}
|
||||
if (file==null)
|
||||
throw new FileNotFoundException(name);
|
||||
|
@ -113,7 +110,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
ensureOpen();
|
||||
RAMFile file;
|
||||
synchronized (this) {
|
||||
file = (RAMFile)fileMap.get(name);
|
||||
file = fileMap.get(name);
|
||||
}
|
||||
if (file==null)
|
||||
throw new FileNotFoundException(name);
|
||||
|
@ -141,7 +138,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
ensureOpen();
|
||||
RAMFile file;
|
||||
synchronized (this) {
|
||||
file = (RAMFile)fileMap.get(name);
|
||||
file = fileMap.get(name);
|
||||
}
|
||||
if (file==null)
|
||||
throw new FileNotFoundException(name);
|
||||
|
@ -161,7 +158,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
*/
|
||||
public synchronized void deleteFile(String name) throws IOException {
|
||||
ensureOpen();
|
||||
RAMFile file = (RAMFile)fileMap.get(name);
|
||||
RAMFile file = fileMap.get(name);
|
||||
if (file!=null) {
|
||||
fileMap.remove(name);
|
||||
file.directory = null;
|
||||
|
@ -175,7 +172,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
ensureOpen();
|
||||
RAMFile file = new RAMFile(this);
|
||||
synchronized (this) {
|
||||
RAMFile existing = (RAMFile)fileMap.get(name);
|
||||
RAMFile existing = fileMap.get(name);
|
||||
if (existing!=null) {
|
||||
sizeInBytes -= existing.sizeInBytes;
|
||||
existing.directory = null;
|
||||
|
@ -190,7 +187,7 @@ public class RAMDirectory extends Directory implements Serializable {
|
|||
ensureOpen();
|
||||
RAMFile file;
|
||||
synchronized (this) {
|
||||
file = (RAMFile)fileMap.get(name);
|
||||
file = fileMap.get(name);
|
||||
}
|
||||
if (file == null)
|
||||
throw new FileNotFoundException(name);
|
||||
|
|
|
@ -24,7 +24,7 @@ class RAMFile implements Serializable {
|
|||
|
||||
private static final long serialVersionUID = 1l;
|
||||
|
||||
private ArrayList buffers = new ArrayList();
|
||||
private ArrayList<byte[]> buffers = new ArrayList<byte[]>();
|
||||
long length;
|
||||
RAMDirectory directory;
|
||||
long sizeInBytes; // Only maintained if in a directory; updates synchronized on directory
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.util.HashSet;
|
|||
|
||||
public class SingleInstanceLockFactory extends LockFactory {
|
||||
|
||||
private HashSet locks = new HashSet();
|
||||
private HashSet<String> locks = new HashSet<String>();
|
||||
|
||||
public Lock makeLock(String lockName) {
|
||||
// We do not use the LockPrefix at all, because the private
|
||||
|
@ -54,9 +54,9 @@ public class SingleInstanceLockFactory extends LockFactory {
|
|||
class SingleInstanceLock extends Lock {
|
||||
|
||||
String lockName;
|
||||
private HashSet locks;
|
||||
private HashSet<String> locks;
|
||||
|
||||
public SingleInstanceLock(HashSet locks, String lockName) {
|
||||
public SingleInstanceLock(HashSet<String> locks, String lockName) {
|
||||
this.locks = locks;
|
||||
this.lockName = lockName;
|
||||
}
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Map;
|
|||
*/
|
||||
public class AverageGuessMemoryModel extends MemoryModel {
|
||||
// best guess primitive sizes
|
||||
private final Map sizes = new IdentityHashMap() {
|
||||
private final Map<Class,Integer> sizes = new IdentityHashMap<Class,Integer>() {
|
||||
{
|
||||
put(boolean.class, Integer.valueOf(1));
|
||||
put(byte.class, Integer.valueOf(1));
|
||||
|
@ -61,7 +61,7 @@ public class AverageGuessMemoryModel extends MemoryModel {
|
|||
* @see org.apache.lucene.util.MemoryModel#getPrimitiveSize(java.lang.Class)
|
||||
*/
|
||||
public int getPrimitiveSize(Class clazz) {
|
||||
return ((Integer) sizes.get(clazz)).intValue();
|
||||
return sizes.get(clazz).intValue();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
|
|
|
@ -19,7 +19,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;
|
||||
import java.util.Set;
|
||||
|
@ -111,13 +110,13 @@ public final class FieldCacheSanityChecker {
|
|||
//
|
||||
// maps the (valId) identityhashCode of cache values to
|
||||
// sets of CacheEntry instances
|
||||
final MapOfSets valIdToItems = new MapOfSets(new HashMap(17));
|
||||
final MapOfSets<Integer, CacheEntry> valIdToItems = new MapOfSets<Integer, CacheEntry>(new HashMap<Integer, Set<CacheEntry>>(17));
|
||||
// maps ReaderField keys to Sets of ValueIds
|
||||
final MapOfSets readerFieldToValIds = new MapOfSets(new HashMap(17));
|
||||
final MapOfSets<ReaderField, Integer> readerFieldToValIds = new MapOfSets<ReaderField, Integer>(new HashMap<ReaderField, Set<Integer>>(17));
|
||||
//
|
||||
|
||||
// any keys that we know result in more then one valId
|
||||
final Set valMismatchKeys = new HashSet();
|
||||
final Set<ReaderField> valMismatchKeys = new HashSet<ReaderField>();
|
||||
|
||||
// iterate over all the cacheEntries to get the mappings we'll need
|
||||
for (int i = 0; i < cacheEntries.length; i++) {
|
||||
|
@ -139,7 +138,7 @@ public final class FieldCacheSanityChecker {
|
|||
}
|
||||
}
|
||||
|
||||
final List insanity = new ArrayList(valMismatchKeys.size() * 3);
|
||||
final List<Insanity> insanity = new ArrayList<Insanity>(valMismatchKeys.size() * 3);
|
||||
|
||||
insanity.addAll(checkValueMismatch(valIdToItems,
|
||||
readerFieldToValIds,
|
||||
|
@ -147,7 +146,7 @@ public final class FieldCacheSanityChecker {
|
|||
insanity.addAll(checkSubreaders(valIdToItems,
|
||||
readerFieldToValIds));
|
||||
|
||||
return (Insanity[]) insanity.toArray(new Insanity[insanity.size()]);
|
||||
return insanity.toArray(new Insanity[insanity.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -157,31 +156,27 @@ public final class FieldCacheSanityChecker {
|
|||
* the Insanity objects.
|
||||
* @see InsanityType#VALUEMISMATCH
|
||||
*/
|
||||
private Collection checkValueMismatch(MapOfSets valIdToItems,
|
||||
MapOfSets readerFieldToValIds,
|
||||
Set valMismatchKeys) {
|
||||
private Collection<Insanity> checkValueMismatch(MapOfSets<Integer, CacheEntry> valIdToItems,
|
||||
MapOfSets<ReaderField, Integer> readerFieldToValIds,
|
||||
Set<ReaderField> valMismatchKeys) {
|
||||
|
||||
final List insanity = new ArrayList(valMismatchKeys.size() * 3);
|
||||
final List<Insanity> insanity = new ArrayList<Insanity>(valMismatchKeys.size() * 3);
|
||||
|
||||
if (! valMismatchKeys.isEmpty() ) {
|
||||
// we have multiple values for some ReaderFields
|
||||
|
||||
final Map rfMap = readerFieldToValIds.getMap();
|
||||
final Map valMap = valIdToItems.getMap();
|
||||
final Iterator mismatchIter = valMismatchKeys.iterator();
|
||||
while (mismatchIter.hasNext()) {
|
||||
final ReaderField rf = (ReaderField)mismatchIter.next();
|
||||
final List badEntries = new ArrayList(valMismatchKeys.size() * 2);
|
||||
final Iterator valIter = ((Set)rfMap.get(rf)).iterator();
|
||||
while (valIter.hasNext()) {
|
||||
Iterator entriesIter = ((Set)valMap.get(valIter.next())).iterator();
|
||||
while (entriesIter.hasNext()) {
|
||||
badEntries.add(entriesIter.next());
|
||||
final Map<ReaderField, Set<Integer>> rfMap = readerFieldToValIds.getMap();
|
||||
final Map<Integer, Set<CacheEntry>> valMap = valIdToItems.getMap();
|
||||
for (final ReaderField rf : valMismatchKeys) {
|
||||
final List<CacheEntry> badEntries = new ArrayList<CacheEntry>(valMismatchKeys.size() * 2);
|
||||
for(final Integer value: rfMap.get(rf)) {
|
||||
for (final CacheEntry cacheEntry : valMap.get(value)) {
|
||||
badEntries.add(cacheEntry);
|
||||
}
|
||||
}
|
||||
|
||||
CacheEntry[] badness = new CacheEntry[badEntries.size()];
|
||||
badness = (CacheEntry[]) badEntries.toArray(badness);
|
||||
badness = badEntries.toArray(badness);
|
||||
|
||||
insanity.add(new Insanity(InsanityType.VALUEMISMATCH,
|
||||
"Multiple distinct value objects for " +
|
||||
|
@ -199,35 +194,33 @@ public final class FieldCacheSanityChecker {
|
|||
*
|
||||
* @see InsanityType#SUBREADER
|
||||
*/
|
||||
private Collection checkSubreaders(MapOfSets valIdToItems,
|
||||
MapOfSets readerFieldToValIds) {
|
||||
private Collection<Insanity> checkSubreaders( MapOfSets<Integer, CacheEntry> valIdToItems,
|
||||
MapOfSets<ReaderField, Integer> readerFieldToValIds) {
|
||||
|
||||
final List insanity = new ArrayList(23);
|
||||
final List<Insanity> insanity = new ArrayList<Insanity>(23);
|
||||
|
||||
Map badChildren = new HashMap(17);
|
||||
MapOfSets badKids = new MapOfSets(badChildren); // wrapper
|
||||
Map<ReaderField, Set<ReaderField>> badChildren = new HashMap<ReaderField, Set<ReaderField>>(17);
|
||||
MapOfSets<ReaderField, ReaderField> badKids = new MapOfSets<ReaderField, ReaderField>(badChildren); // wrapper
|
||||
|
||||
Map viToItemSets = valIdToItems.getMap();
|
||||
Map rfToValIdSets = readerFieldToValIds.getMap();
|
||||
Map<Integer, Set<CacheEntry>> viToItemSets = valIdToItems.getMap();
|
||||
Map<ReaderField, Set<Integer>> rfToValIdSets = readerFieldToValIds.getMap();
|
||||
|
||||
Set seen = new HashSet(17);
|
||||
Set<ReaderField> seen = new HashSet<ReaderField>(17);
|
||||
|
||||
Set readerFields = rfToValIdSets.keySet();
|
||||
Iterator rfIter = readerFields.iterator();
|
||||
while (rfIter.hasNext()) {
|
||||
ReaderField rf = (ReaderField) rfIter.next();
|
||||
Set<ReaderField> readerFields = rfToValIdSets.keySet();
|
||||
for (final ReaderField rf : readerFields) {
|
||||
|
||||
if (seen.contains(rf)) continue;
|
||||
|
||||
List kids = getAllDecendentReaderKeys(rf.readerKey);
|
||||
for (int i = 0; i < kids.size(); i++) {
|
||||
ReaderField kid = new ReaderField(kids.get(i), rf.fieldName);
|
||||
for (Object kidKey : kids) {
|
||||
ReaderField kid = new ReaderField(kidKey, rf.fieldName);
|
||||
|
||||
if (badChildren.containsKey(kid)) {
|
||||
// we've already process this kid as RF and found other problems
|
||||
// track those problems as our own
|
||||
badKids.put(rf, kid);
|
||||
badKids.putAll(rf, (Collection)badChildren.get(kid));
|
||||
badKids.putAll(rf, badChildren.get(kid));
|
||||
badChildren.remove(kid);
|
||||
|
||||
} else if (rfToValIdSets.containsKey(kid)) {
|
||||
|
@ -240,33 +233,27 @@ public final class FieldCacheSanityChecker {
|
|||
}
|
||||
|
||||
// every mapping in badKids represents an Insanity
|
||||
Iterator parentsIter = badChildren.keySet().iterator();
|
||||
while (parentsIter.hasNext()) {
|
||||
ReaderField parent = (ReaderField) parentsIter.next();
|
||||
Set kids = (Set) badChildren.get(parent);
|
||||
for (final ReaderField parent : badChildren.keySet()) {
|
||||
Set<ReaderField> kids = badChildren.get(parent);
|
||||
|
||||
List badEntries = new ArrayList(kids.size() * 2);
|
||||
List<CacheEntry> badEntries = new ArrayList<CacheEntry>(kids.size() * 2);
|
||||
|
||||
// put parent entr(ies) in first
|
||||
{
|
||||
Iterator valIter =((Set)rfToValIdSets.get(parent)).iterator();
|
||||
while (valIter.hasNext()) {
|
||||
badEntries.addAll((Set)viToItemSets.get(valIter.next()));
|
||||
for (final Integer value : rfToValIdSets.get(parent)) {
|
||||
badEntries.addAll(viToItemSets.get(value));
|
||||
}
|
||||
}
|
||||
|
||||
// now the entries for the descendants
|
||||
Iterator kidsIter = kids.iterator();
|
||||
while (kidsIter.hasNext()) {
|
||||
ReaderField kid = (ReaderField) kidsIter.next();
|
||||
Iterator valIter =((Set)rfToValIdSets.get(kid)).iterator();
|
||||
while (valIter.hasNext()) {
|
||||
badEntries.addAll((Set)viToItemSets.get(valIter.next()));
|
||||
for (final ReaderField kid : kids) {
|
||||
for (final Integer value : rfToValIdSets.get(kid)) {
|
||||
badEntries.addAll(viToItemSets.get(value));
|
||||
}
|
||||
}
|
||||
|
||||
CacheEntry[] badness = new CacheEntry[badEntries.size()];
|
||||
badness = (CacheEntry[]) badEntries.toArray(badness);
|
||||
badness = badEntries.toArray(badness);
|
||||
|
||||
insanity.add(new Insanity(InsanityType.SUBREADER,
|
||||
"Found caches for decendents of " +
|
||||
|
|
|
@ -26,21 +26,21 @@ import java.util.Map;
|
|||
/**
|
||||
* Helper class for keeping Lists of Objects associated with keys. <b>WARNING: THIS CLASS IS NOT THREAD SAFE</b>
|
||||
*/
|
||||
public class MapOfSets {
|
||||
public class MapOfSets<K, V> {
|
||||
|
||||
private final Map theMap;
|
||||
private final Map<K, Set<V>> theMap;
|
||||
|
||||
/**
|
||||
* @param m the backing store for this object
|
||||
*/
|
||||
public MapOfSets(Map m) {
|
||||
public MapOfSets(Map<K, Set<V>> m) {
|
||||
theMap = m;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return direct access to the map backing this object.
|
||||
*/
|
||||
public Map getMap() {
|
||||
public Map<K, Set<V>> getMap() {
|
||||
return theMap;
|
||||
}
|
||||
|
||||
|
@ -49,12 +49,12 @@ public class MapOfSets {
|
|||
* already in the map, a new Set will first be created.
|
||||
* @return the size of the Set associated with key once val is added to it.
|
||||
*/
|
||||
public int put(Object key, Object val) {
|
||||
final Set theSet;
|
||||
public int put(K key, V val) {
|
||||
final Set<V> theSet;
|
||||
if (theMap.containsKey(key)) {
|
||||
theSet = (Set)theMap.get(key);
|
||||
theSet = theMap.get(key);
|
||||
} else {
|
||||
theSet = new HashSet(23);
|
||||
theSet = new HashSet<V>(23);
|
||||
theMap.put(key, theSet);
|
||||
}
|
||||
theSet.add(val);
|
||||
|
@ -66,12 +66,12 @@ public class MapOfSets {
|
|||
* already in the map, a new Set will first be created.
|
||||
* @return the size of the Set associated with key once val is added to it.
|
||||
*/
|
||||
public int putAll(Object key, Collection vals) {
|
||||
final Set theSet;
|
||||
public int putAll(K key, Collection<? extends V> vals) {
|
||||
final Set<V> theSet;
|
||||
if (theMap.containsKey(key)) {
|
||||
theSet = (Set)theMap.get(key);
|
||||
theSet = theMap.get(key);
|
||||
} else {
|
||||
theSet = new HashSet(23);
|
||||
theSet = new HashSet<V>(23);
|
||||
theMap.put(key, theSet);
|
||||
}
|
||||
theSet.addAll(vals);
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.util.Map;
|
|||
*/
|
||||
public abstract class Parameter implements Serializable
|
||||
{
|
||||
static Map allParameters = new HashMap();
|
||||
static Map<String,Parameter> allParameters = new HashMap<String,Parameter>();
|
||||
|
||||
private String name;
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import java.util.*;
|
|||
public final class RamUsageEstimator {
|
||||
private MemoryModel memoryModel;
|
||||
|
||||
private final Map seen;
|
||||
private final Map<Object,Object> seen;
|
||||
|
||||
private int refSize;
|
||||
private int arraySize;
|
||||
|
@ -82,7 +82,7 @@ public final class RamUsageEstimator {
|
|||
this.checkInterned = checkInterned;
|
||||
// Use Map rather than Set so that we can use an IdentityHashMap - not
|
||||
// seeing an IdentityHashSet
|
||||
seen = new IdentityHashMap(64);
|
||||
seen = new IdentityHashMap<Object,Object>(64);
|
||||
this.refSize = memoryModel.getReferenceSize();
|
||||
this.arraySize = memoryModel.getArraySize();
|
||||
this.classSize = memoryModel.getClassSize();
|
||||
|
|
|
@ -34,7 +34,7 @@ public class ReaderUtil {
|
|||
* @param allSubReaders
|
||||
* @param reader
|
||||
*/
|
||||
public static void gatherSubReaders(List allSubReaders, IndexReader reader) {
|
||||
public static void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader reader) {
|
||||
IndexReader[] subReaders = reader.getSequentialSubReaders();
|
||||
if (subReaders == null) {
|
||||
// Add the reader itself, and do not recurse
|
||||
|
@ -54,7 +54,7 @@ public class ReaderUtil {
|
|||
* @return sub reader of parent which contains the specified doc id
|
||||
*/
|
||||
public static IndexReader subReader(int doc, IndexReader reader) {
|
||||
List subReadersList = new ArrayList();
|
||||
List<IndexReader> subReadersList = new ArrayList<IndexReader>();
|
||||
ReaderUtil.gatherSubReaders(subReadersList, reader);
|
||||
IndexReader[] subReaders = (IndexReader[]) subReadersList
|
||||
.toArray(new IndexReader[subReadersList.size()]);
|
||||
|
@ -75,7 +75,7 @@ public class ReaderUtil {
|
|||
* @return the subreader at subIndex
|
||||
*/
|
||||
public static IndexReader subReader(IndexReader reader, int subIndex) {
|
||||
List subReadersList = new ArrayList();
|
||||
List<IndexReader> subReadersList = new ArrayList<IndexReader>();
|
||||
ReaderUtil.gatherSubReaders(subReadersList, reader);
|
||||
IndexReader[] subReaders = (IndexReader[]) subReadersList
|
||||
.toArray(new IndexReader[subReadersList.size()]);
|
||||
|
|
|
@ -21,43 +21,48 @@ package org.apache.lucene.util.cache;
|
|||
/**
|
||||
* Base class for cache implementations.
|
||||
*/
|
||||
public abstract class Cache {
|
||||
public abstract class Cache<K,V> {
|
||||
|
||||
/**
|
||||
* Simple Cache wrapper that synchronizes all
|
||||
* calls that access the cache.
|
||||
*/
|
||||
static class SynchronizedCache extends Cache {
|
||||
Object mutex;
|
||||
Cache cache;
|
||||
static class SynchronizedCache<K,V> extends Cache<K,V> {
|
||||
private Object mutex;
|
||||
private Cache<K,V> cache;
|
||||
|
||||
SynchronizedCache(Cache cache) {
|
||||
SynchronizedCache(Cache<K,V> cache) {
|
||||
this.cache = cache;
|
||||
this.mutex = this;
|
||||
}
|
||||
|
||||
SynchronizedCache(Cache cache, Object mutex) {
|
||||
SynchronizedCache(Cache<K,V> cache, Object mutex) {
|
||||
this.cache = cache;
|
||||
this.mutex = mutex;
|
||||
}
|
||||
|
||||
public void put(Object key, Object value) {
|
||||
@Override
|
||||
public void put(K key, V value) {
|
||||
synchronized(mutex) {cache.put(key, value);}
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
@Override
|
||||
public V get(Object key) {
|
||||
synchronized(mutex) {return cache.get(key);}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
synchronized(mutex) {return cache.containsKey(key);}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
synchronized(mutex) {cache.close();}
|
||||
}
|
||||
|
||||
Cache getSynchronizedCache() {
|
||||
@Override
|
||||
Cache<K,V> getSynchronizedCache() {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
@ -67,7 +72,7 @@ public abstract class Cache {
|
|||
* In order to guarantee thread-safety, all access to the backed cache must
|
||||
* be accomplished through the returned cache.
|
||||
*/
|
||||
public static Cache synchronizedCache(Cache cache) {
|
||||
public static <K,V> Cache<K,V> synchronizedCache(Cache<K,V> cache) {
|
||||
return cache.getSynchronizedCache();
|
||||
}
|
||||
|
||||
|
@ -78,19 +83,19 @@ public abstract class Cache {
|
|||
* e. g. subclasses of {@link SynchronizedCache} or this
|
||||
* in case this cache is already synchronized.
|
||||
*/
|
||||
Cache getSynchronizedCache() {
|
||||
return new SynchronizedCache(this);
|
||||
Cache<K,V> getSynchronizedCache() {
|
||||
return new SynchronizedCache<K,V>(this);
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts a (key, value)-pair into the cache.
|
||||
*/
|
||||
public abstract void put(Object key, Object value);
|
||||
public abstract void put(K key, V value);
|
||||
|
||||
/**
|
||||
* Returns the value for the given key.
|
||||
*/
|
||||
public abstract Object get(Object key);
|
||||
public abstract V get(Object key);
|
||||
|
||||
/**
|
||||
* Returns whether the given key is in this cache.
|
||||
|
|
|
@ -26,24 +26,19 @@ import java.util.Map;
|
|||
* if needed.
|
||||
*
|
||||
*/
|
||||
public class SimpleLRUCache extends SimpleMapCache {
|
||||
public class SimpleLRUCache<K,V> extends SimpleMapCache<K,V> {
|
||||
private final static float LOADFACTOR = 0.75f;
|
||||
|
||||
private int cacheSize;
|
||||
|
||||
/**
|
||||
* Creates a last-recently-used cache with the specified size.
|
||||
*/
|
||||
public SimpleLRUCache(int cacheSize) {
|
||||
super(null);
|
||||
this.cacheSize = cacheSize;
|
||||
int capacity = (int) Math.ceil(cacheSize / LOADFACTOR) + 1;
|
||||
|
||||
super.map = new LinkedHashMap(capacity, LOADFACTOR, true) {
|
||||
protected boolean removeEldestEntry(Map.Entry eldest) {
|
||||
return size() > SimpleLRUCache.this.cacheSize;
|
||||
public SimpleLRUCache(final int cacheSize) {
|
||||
super(new LinkedHashMap<K,V>((int) Math.ceil(cacheSize / LOADFACTOR) + 1, LOADFACTOR, true) {
|
||||
@Override
|
||||
protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
|
||||
return size() > cacheSize;
|
||||
}
|
||||
};
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -26,29 +26,33 @@ import java.util.Set;
|
|||
* This cache is not synchronized, use {@link Cache#synchronizedCache(Cache)}
|
||||
* if needed.
|
||||
*/
|
||||
public class SimpleMapCache extends Cache {
|
||||
Map map;
|
||||
public class SimpleMapCache<K,V> extends Cache<K,V> {
|
||||
protected Map<K,V> map;
|
||||
|
||||
public SimpleMapCache() {
|
||||
this(new HashMap());
|
||||
this(new HashMap<K,V>());
|
||||
}
|
||||
|
||||
public SimpleMapCache(Map map) {
|
||||
public SimpleMapCache(Map<K,V> map) {
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
@Override
|
||||
public V get(Object key) {
|
||||
return map.get(key);
|
||||
}
|
||||
|
||||
public void put(Object key, Object value) {
|
||||
@Override
|
||||
public void put(K key, V value) {
|
||||
map.put(key, value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
// NOOP
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
return map.containsKey(key);
|
||||
}
|
||||
|
@ -56,44 +60,51 @@ public class SimpleMapCache extends Cache {
|
|||
/**
|
||||
* Returns a Set containing all keys in this cache.
|
||||
*/
|
||||
public Set keySet() {
|
||||
public Set<K> keySet() {
|
||||
return map.keySet();
|
||||
}
|
||||
|
||||
Cache getSynchronizedCache() {
|
||||
return new SynchronizedSimpleMapCache(this);
|
||||
@Override
|
||||
Cache<K,V> getSynchronizedCache() {
|
||||
return new SynchronizedSimpleMapCache<K,V>(this);
|
||||
}
|
||||
|
||||
private static class SynchronizedSimpleMapCache extends SimpleMapCache {
|
||||
Object mutex;
|
||||
SimpleMapCache cache;
|
||||
private static class SynchronizedSimpleMapCache<K,V> extends SimpleMapCache<K,V> {
|
||||
private Object mutex;
|
||||
private SimpleMapCache<K,V> cache;
|
||||
|
||||
SynchronizedSimpleMapCache(SimpleMapCache cache) {
|
||||
SynchronizedSimpleMapCache(SimpleMapCache<K,V> cache) {
|
||||
this.cache = cache;
|
||||
this.mutex = this;
|
||||
}
|
||||
|
||||
public void put(Object key, Object value) {
|
||||
@Override
|
||||
public void put(K key, V value) {
|
||||
synchronized(mutex) {cache.put(key, value);}
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
@Override
|
||||
public V get(Object key) {
|
||||
synchronized(mutex) {return cache.get(key);}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
synchronized(mutex) {return cache.containsKey(key);}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
synchronized(mutex) {cache.close();}
|
||||
}
|
||||
|
||||
public Set keySet() {
|
||||
@Override
|
||||
public Set<K> keySet() {
|
||||
synchronized(mutex) {return cache.keySet();}
|
||||
}
|
||||
|
||||
Cache getSynchronizedCache() {
|
||||
@Override
|
||||
Cache<K,V> getSynchronizedCache() {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -335,7 +335,7 @@ public class TestIndexReaderClone extends LuceneTestCase {
|
|||
origSegmentReader.close();
|
||||
assertDelDocsRefCountEquals(1, origSegmentReader);
|
||||
// check the norm refs
|
||||
Norm norm = (Norm) clonedSegmentReader.norms.get("field1");
|
||||
Norm norm = clonedSegmentReader.norms.get("field1");
|
||||
assertEquals(1, norm.bytesRef().refCount());
|
||||
clonedSegmentReader.close();
|
||||
dir1.close();
|
||||
|
|
|
@ -170,7 +170,7 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
TestIndexReaderReopen.createIndex(dir1, false);
|
||||
SegmentReader reader1 = SegmentReader.getOnlySegmentReader(dir1);
|
||||
reader1.norms("field1");
|
||||
Norm r1norm = (Norm)reader1.norms.get("field1");
|
||||
Norm r1norm = reader1.norms.get("field1");
|
||||
SegmentReader.Ref r1BytesRef = r1norm.bytesRef();
|
||||
SegmentReader reader2 = (SegmentReader)reader1.clone();
|
||||
assertEquals(2, r1norm.bytesRef().refCount());
|
||||
|
@ -189,14 +189,14 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
IndexReader reader2C = (IndexReader) reader1.clone();
|
||||
SegmentReader segmentReader2C = SegmentReader.getOnlySegmentReader(reader2C);
|
||||
segmentReader2C.norms("field1"); // load the norms for the field
|
||||
Norm reader2CNorm = (Norm)segmentReader2C.norms.get("field1");
|
||||
Norm reader2CNorm = segmentReader2C.norms.get("field1");
|
||||
assertTrue("reader2CNorm.bytesRef()=" + reader2CNorm.bytesRef(), reader2CNorm.bytesRef().refCount() == 2);
|
||||
|
||||
|
||||
|
||||
IndexReader reader3C = (IndexReader) reader2C.clone();
|
||||
SegmentReader segmentReader3C = SegmentReader.getOnlySegmentReader(reader3C);
|
||||
Norm reader3CCNorm = (Norm)segmentReader3C.norms.get("field1");
|
||||
Norm reader3CCNorm = segmentReader3C.norms.get("field1");
|
||||
assertEquals(3, reader3CCNorm.bytesRef().refCount());
|
||||
|
||||
// edit a norm and the refcount should be 1
|
||||
|
@ -215,13 +215,13 @@ public class TestIndexReaderCloneNorms extends LuceneTestCase {
|
|||
|
||||
// norm values should be different
|
||||
assertTrue(Similarity.decodeNorm(segmentReader3C.norms("field1")[5]) != Similarity.decodeNorm(segmentReader4C.norms("field1")[5]));
|
||||
Norm reader4CCNorm = (Norm)segmentReader4C.norms.get("field1");
|
||||
Norm reader4CCNorm = segmentReader4C.norms.get("field1");
|
||||
assertEquals(3, reader3CCNorm.bytesRef().refCount());
|
||||
assertEquals(1, reader4CCNorm.bytesRef().refCount());
|
||||
|
||||
IndexReader reader5C = (IndexReader) reader4C.clone();
|
||||
SegmentReader segmentReader5C = SegmentReader.getOnlySegmentReader(reader5C);
|
||||
Norm reader5CCNorm = (Norm)segmentReader5C.norms.get("field1");
|
||||
Norm reader5CCNorm = segmentReader5C.norms.get("field1");
|
||||
reader5C.setNorm(5, "field1", 0.7f);
|
||||
assertEquals(1, reader5CCNorm.bytesRef().refCount());
|
||||
|
||||
|
|
|
@ -43,19 +43,19 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
Random randomState;
|
||||
boolean noDeleteOpenFile = true;
|
||||
boolean preventDoubleWrite = true;
|
||||
private Set unSyncedFiles;
|
||||
private Set createdFiles;
|
||||
private Set<String> unSyncedFiles;
|
||||
private Set<String> createdFiles;
|
||||
volatile boolean crashed;
|
||||
|
||||
// NOTE: we cannot initialize the Map here due to the
|
||||
// order in which our constructor actually does this
|
||||
// member initialization vs when it calls super. It seems
|
||||
// like super is called, then our members are initialized:
|
||||
Map openFiles;
|
||||
Map<String,Integer> openFiles;
|
||||
|
||||
private synchronized void init() {
|
||||
if (openFiles == null)
|
||||
openFiles = new HashMap();
|
||||
openFiles = new HashMap<String,Integer>();
|
||||
if (createdFiles == null)
|
||||
createdFiles = new HashSet();
|
||||
if (unSyncedFiles == null)
|
||||
|
@ -90,12 +90,12 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
public synchronized void crash() throws IOException {
|
||||
crashed = true;
|
||||
openFiles = new HashMap();
|
||||
Iterator it = unSyncedFiles.iterator();
|
||||
Iterator<String> it = unSyncedFiles.iterator();
|
||||
unSyncedFiles = new HashSet();
|
||||
int count = 0;
|
||||
while(it.hasNext()) {
|
||||
String name = (String) it.next();
|
||||
RAMFile file = (RAMFile) fileMap.get(name);
|
||||
String name = it.next();
|
||||
RAMFile file = fileMap.get(name);
|
||||
if (count % 3 == 0) {
|
||||
deleteFile(name, true);
|
||||
} else if (count % 3 == 1) {
|
||||
|
@ -204,7 +204,7 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
throw new IOException("cannot createOutput after crash");
|
||||
unSyncedFiles.add(name);
|
||||
createdFiles.add(name);
|
||||
RAMFile existing = (RAMFile)fileMap.get(name);
|
||||
RAMFile existing = fileMap.get(name);
|
||||
// Enforce write once:
|
||||
if (existing!=null && !name.equals("segments.gen") && preventDoubleWrite)
|
||||
throw new IOException("file " + name + " already exists");
|
||||
|
@ -221,7 +221,7 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
}
|
||||
|
||||
public synchronized IndexInput openInput(String name) throws IOException {
|
||||
RAMFile file = (RAMFile)fileMap.get(name);
|
||||
RAMFile file = fileMap.get(name);
|
||||
if (file == null)
|
||||
throw new FileNotFoundException(name);
|
||||
else {
|
||||
|
@ -239,9 +239,9 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
/** Provided for testing purposes. Use sizeInBytes() instead. */
|
||||
public synchronized final long getRecomputedSizeInBytes() {
|
||||
long size = 0;
|
||||
Iterator it = fileMap.values().iterator();
|
||||
while (it.hasNext())
|
||||
size += ((RAMFile) it.next()).getSizeInBytes();
|
||||
for(final RAMFile file: fileMap.values()) {
|
||||
size += file.getSizeInBytes();
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
|
@ -253,9 +253,8 @@ public class MockRAMDirectory extends RAMDirectory {
|
|||
|
||||
public final synchronized long getRecomputedActualSizeInBytes() {
|
||||
long size = 0;
|
||||
Iterator it = fileMap.values().iterator();
|
||||
while (it.hasNext())
|
||||
size += ((RAMFile) it.next()).length;
|
||||
for (final RAMFile file : fileMap.values())
|
||||
size += file.length;
|
||||
return size;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue